lightstream 0.4.3

Composable, zero-copy Arrow IPC and native data streaming for Rust with SIMD-aligned I/O, async support, and memory-mapping.
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
#[cfg(test)]
mod pyarrow_roundtrip_tests {
    use std::sync::Arc;

    use ::lightstream::enums::IPCMessageProtocol;
    use ::lightstream::models::readers::ipc::file_table_reader::FileTableReader;
    use ::lightstream::models::readers::ipc::table_stream_reader::TableStreamReader;
    use ::lightstream::models::streams::disk::DiskByteStream;
    use ::lightstream::models::writers::ipc::table_stream_writer::write_tables_to_stream;
    use ::lightstream::models::writers::ipc::table_writer::write_tables_to_file;
    use futures_util::stream::StreamExt;
    use minarrow::ffi::arrow_dtype::ArrowType;
    use minarrow::*;
    use minarrow::{Array, TextArray, Vec64};

    /// Validate that data values match between expected and actual tables
    fn validate_table_data(expected: &Table, actual: &Table) {
        assert_eq!(expected.n_rows, actual.n_rows, "Row count mismatch");
        assert_eq!(
            expected.cols.len(),
            actual.cols.len(),
            "Column count mismatch"
        );

        for (i, (expected_col, actual_col)) in
            expected.cols.iter().zip(actual.cols.iter()).enumerate()
        {
            println!("Validating column {}: {}", i, expected_col.field.name);

            // Check field metadata
            assert_eq!(
                expected_col.field.name, actual_col.field.name,
                "Column {} name mismatch",
                i
            );
            assert_eq!(
                expected_col.field.dtype, actual_col.field.dtype,
                "Column {} type mismatch",
                i
            );

            // Check array data based on type
            match (&expected_col.array, &actual_col.array) {
                (
                    Array::NumericArray(NumericArray::Int32(exp)),
                    Array::NumericArray(NumericArray::Int32(act)),
                ) => {
                    assert_eq!(
                        exp.data.as_slice(),
                        act.data.as_slice(),
                        "Int32 data mismatch in column {}",
                        i
                    );
                }
                (
                    Array::NumericArray(NumericArray::Int64(exp)),
                    Array::NumericArray(NumericArray::Int64(act)),
                ) => {
                    assert_eq!(
                        exp.data.as_slice(),
                        act.data.as_slice(),
                        "Int64 data mismatch in column {}",
                        i
                    );
                }
                (
                    Array::NumericArray(NumericArray::UInt32(exp)),
                    Array::NumericArray(NumericArray::UInt32(act)),
                ) => {
                    assert_eq!(
                        exp.data.as_slice(),
                        act.data.as_slice(),
                        "UInt32 data mismatch in column {}",
                        i
                    );
                }
                (
                    Array::NumericArray(NumericArray::UInt64(exp)),
                    Array::NumericArray(NumericArray::UInt64(act)),
                ) => {
                    assert_eq!(
                        exp.data.as_slice(),
                        act.data.as_slice(),
                        "UInt64 data mismatch in column {}",
                        i
                    );
                }
                (
                    Array::NumericArray(NumericArray::Float32(exp)),
                    Array::NumericArray(NumericArray::Float32(act)),
                ) => {
                    for (e, a) in exp.data.iter().zip(act.data.iter()) {
                        assert!(
                            (e - a).abs() < f32::EPSILON,
                            "Float32 data mismatch in column {}: {} != {}",
                            i,
                            e,
                            a
                        );
                    }
                }
                (
                    Array::NumericArray(NumericArray::Float64(exp)),
                    Array::NumericArray(NumericArray::Float64(act)),
                ) => {
                    for (e, a) in exp.data.iter().zip(act.data.iter()) {
                        assert!(
                            (e - a).abs() < f64::EPSILON,
                            "Float64 data mismatch in column {}: {} != {}",
                            i,
                            e,
                            a
                        );
                    }
                }
                (Array::BooleanArray(exp), Array::BooleanArray(act)) => {
                    assert_eq!(
                        exp.len, act.len,
                        "Boolean array length mismatch in column {}",
                        i
                    );
                    for j in 0..exp.len {
                        assert_eq!(
                            exp.get(j),
                            act.get(j),
                            "Boolean data mismatch at index {} in column {}",
                            j,
                            i
                        );
                    }
                }
                (
                    Array::TextArray(TextArray::String32(exp)),
                    Array::TextArray(TextArray::String32(act)),
                ) => {
                    let exp_strings: Vec<String> = exp.iter().map(|s| s.to_string()).collect();
                    let act_strings: Vec<String> = act.iter().map(|s| s.to_string()).collect();
                    assert_eq!(
                        exp_strings, act_strings,
                        "String32 data mismatch in column {}",
                        i
                    );
                }
                (
                    Array::TextArray(TextArray::Categorical32(exp)),
                    Array::TextArray(TextArray::Categorical32(act)),
                ) => {
                    assert_eq!(
                        exp.data.as_slice(),
                        act.data.as_slice(),
                        "Categorical32 indices mismatch in column {}",
                        i
                    );
                    assert_eq!(
                        exp.unique_values, act.unique_values,
                        "Categorical32 values mismatch in column {}",
                        i
                    );
                }
                _ => panic!(
                    "Unsupported array type combination for column {} validation: {:?} vs {:?}",
                    i, expected_col.field.dtype, actual_col.field.dtype
                ),
            }
        }
    }

    /// Create the expected table that matches PyArrow-generated data (basic types only)
    fn create_expected_table(n_rows: usize) -> Table {
        let int32 = NumericArray::Int32(Arc::new(IntegerArray::from_vec64(
            Vec64::from_slice(&(1..=n_rows as i32).collect::<Vec<_>>()),
            None,
        )));
        let int64 = NumericArray::Int64(Arc::new(IntegerArray::from_vec64(
            Vec64::from_slice(&(100..(100 + n_rows as i64)).collect::<Vec<_>>()),
            None,
        )));
        let uint32 = NumericArray::UInt32(Arc::new(IntegerArray::from_vec64(
            Vec64::from_slice(&(0..n_rows as u32).collect::<Vec<_>>()),
            None,
        )));
        let uint64 = NumericArray::UInt64(Arc::new(IntegerArray::from_vec64(
            Vec64::from_slice(&(10..(10 + n_rows as u64)).collect::<Vec<_>>()),
            None,
        )));
        let float32 = NumericArray::Float32(Arc::new(FloatArray::from_vec64(
            Vec64::from_slice(
                &(0..n_rows)
                    .map(|i| i as f32 * 1.25 - 2.5)
                    .collect::<Vec<_>>(),
            ),
            None,
        )));
        let float64 = NumericArray::Float64(Arc::new(FloatArray::from_vec64(
            Vec64::from_slice(
                &(0..n_rows)
                    .map(|i| i as f64 * 3.5 - 1.0)
                    .collect::<Vec<_>>(),
            ),
            None,
        )));
        let bools = BooleanArray::from_slice(&(0..n_rows).map(|i| i % 2 == 0).collect::<Vec<_>>());

        let strs: Vec<String> = (0..n_rows).map(|i| format!("str{i}")).collect();
        let str_refs: Vec<&str> = strs.iter().map(|s| &**s).collect();
        let string32 = StringArray::from_vec(str_refs, None);

        let cols = vec![
            FieldArray::new(
                Field::new("int32", ArrowType::Int32, false, None),
                Array::NumericArray(int32),
            ),
            FieldArray::new(
                Field::new("int64", ArrowType::Int64, false, None),
                Array::NumericArray(int64),
            ),
            FieldArray::new(
                Field::new("uint32", ArrowType::UInt32, false, None),
                Array::NumericArray(uint32),
            ),
            FieldArray::new(
                Field::new("uint64", ArrowType::UInt64, false, None),
                Array::NumericArray(uint64),
            ),
            FieldArray::new(
                Field::new("float32", ArrowType::Float32, false, None),
                Array::NumericArray(float32),
            ),
            FieldArray::new(
                Field::new("float64", ArrowType::Float64, false, None),
                Array::NumericArray(float64),
            ),
            FieldArray::new(
                Field::new("bool", ArrowType::Boolean, false, None),
                Array::BooleanArray(Arc::new(bools)),
            ),
            FieldArray::new(
                Field::new("string", ArrowType::String, false, None),
                Array::TextArray(TextArray::String32(Arc::new(string32))),
            ),
        ];

        Table {
            cols,
            n_rows: n_rows,
            name: "test".to_owned(),
        }
    }

    #[tokio::test]
    async fn test_read_pyarrow_file_format() {
        println!("Testing PyArrow -> lightstream roundtrip (file format)");

        // Read PyArrow-generated Arrow file
        let file_path = "python/pyarrow_basic_types.arrow";
        let reader = FileTableReader::open(file_path).expect("Failed to open file");

        // Read the first (and likely only) table
        assert!(reader.num_batches() > 0, "No record batches found");
        let actual_table = reader.read_batch(0).expect("Failed to read first batch");

        println!(
            "Read table with {} rows and {} columns",
            actual_table.n_rows,
            actual_table.cols.len()
        );

        // Create expected table with matching data
        let expected_table = create_expected_table(4);

        // Validate the data matches
        validate_table_data(&expected_table, &actual_table);

        println!("✅ PyArrow file format read successfully!");
    }

    #[tokio::test]
    async fn test_read_pyarrow_stream_format() {
        println!("Testing PyArrow -> lightstream roundtrip (stream format)");

        // Read PyArrow-generated Arrow stream.
        // PyArrow writes standard 8-byte aligned Arrow IPC, so we use
        // TableStreamReader (Vec<u8>, 8-byte alignment) rather than
        // TableStreamReader64 (Vec64<u8>, 64-byte alignment).
        // DiskByteStream yields Vec64 for I/O buffer alignment but
        // implements AsyncRead, so we wrap it in an adapter that
        // re-chunks into Vec<u8>.
        use std::pin::Pin;
        use std::task::{Context, Poll};
        use tokio::io::{AsyncRead, AsyncReadExt};

        struct ByteStream<R>(R);
        impl<R: AsyncRead + Unpin> futures_core::Stream for ByteStream<R> {
            type Item = Result<Vec<u8>, std::io::Error>;
            fn poll_next(
                mut self: Pin<&mut Self>,
                cx: &mut Context<'_>,
            ) -> Poll<Option<Self::Item>> {
                let mut buf = vec![0u8; 8192];
                let mut fut = Box::pin(self.0.read(&mut buf));
                match fut.as_mut().poll(cx) {
                    Poll::Ready(Ok(0)) => Poll::Ready(None),
                    Poll::Ready(Ok(n)) => {
                        buf.truncate(n);
                        Poll::Ready(Some(Ok(buf)))
                    }
                    Poll::Ready(Err(e)) => Poll::Ready(Some(Err(e))),
                    Poll::Pending => Poll::Pending,
                }
            }
        }
        impl<R: AsyncRead + Unpin> AsyncRead for ByteStream<R> {
            fn poll_read(
                mut self: Pin<&mut Self>,
                cx: &mut Context<'_>,
                buf: &mut tokio::io::ReadBuf<'_>,
            ) -> Poll<std::io::Result<()>> {
                Pin::new(&mut self.0).poll_read(cx, buf)
            }
        }

        let file_path = "python/pyarrow_basic_types.stream";
        use ::lightstream::enums::BufferChunkSize;
        let disk = DiskByteStream::open(file_path, BufferChunkSize::Custom(1024))
            .await
            .expect("Failed to create stream");
        let stream = ByteStream(disk);
        let mut reader = TableStreamReader::new(stream, 1024, IPCMessageProtocol::Stream);

        let mut tables = vec![];
        while let Some(table) = reader.next().await {
            let table = table.expect("Failed to read table from stream");
            tables.push(table);
        }

        assert_eq!(tables.len(), 1, "Expected exactly one table");
        let actual_table = &tables[0];

        println!(
            "Read table with {} rows and {} columns",
            actual_table.n_rows,
            actual_table.cols.len()
        );

        // Create expected table with matching data
        let expected_table = create_expected_table(4);

        // Validate the data matches
        validate_table_data(&expected_table, actual_table);

        println!("✅ PyArrow stream format read successfully!");
    }

    #[tokio::test]
    async fn test_write_lightstream_file_format() {
        println!("Testing lightstream -> PyArrow roundtrip (file format)");

        // Create test data with lightstream
        let test_table = create_expected_table(4);
        let schema = test_table
            .cols
            .iter()
            .map(|col| (*col.field).clone())
            .collect::<Vec<_>>();

        // Write with lightstream to file format
        let file_path = "python/lightstream_basic_types.arrow";
        write_tables_to_file(file_path, &[test_table], schema)
            .await
            .expect("Failed to write table to file");

        println!("Wrote lightstream file to: {}", file_path);
        println!("✅ lightstream file format written successfully!");
        println!("Run `python3 python/validate_lightstream_output.py` to validate with PyArrow");
    }

    #[tokio::test]
    async fn test_write_lightstream_stream_format() {
        println!("Testing lightstream -> PyArrow roundtrip (stream format)");

        // Create test data with lightstream
        let test_table = create_expected_table(4);
        let schema = test_table
            .cols
            .iter()
            .map(|col| (*col.field).clone())
            .collect::<Vec<_>>();

        // Write with lightstream to stream format
        let file_path = "python/lightstream_basic_types.stream";
        let file = tokio::fs::File::create(file_path)
            .await
            .expect("Failed to create stream file");

        write_tables_to_stream::<_, Vec64<u8>>(
            file,
            &[test_table],
            schema,
            IPCMessageProtocol::Stream,
        )
        .await
        .expect("Failed to write table to stream");

        println!("Wrote lightstream stream to: {}", file_path);
        println!("✅ lightstream stream format written successfully!");
        println!("Run `python3 python/validate_lightstream_output.py` to validate with PyArrow");
    }
}