Skip to main content

vortex_layout/layouts/flat/
writer.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright the Vortex contributors
3
4use async_trait::async_trait;
5use futures::StreamExt;
6use vortex_array::ArrayContext;
7use vortex_array::ArrayId;
8use vortex_array::dtype::DType;
9use vortex_array::expr::stats::Precision;
10use vortex_array::expr::stats::Stat;
11use vortex_array::expr::stats::StatsProvider;
12use vortex_array::normalize::NormalizeOptions;
13use vortex_array::normalize::Operation;
14use vortex_array::scalar::Scalar;
15use vortex_array::scalar::ScalarTruncation;
16use vortex_array::scalar::lower_bound;
17use vortex_array::scalar::upper_bound;
18use vortex_array::serde::SerializeOptions;
19use vortex_array::stats::StatsSetRef;
20use vortex_buffer::BufferString;
21use vortex_buffer::ByteBuffer;
22use vortex_error::VortexExpect;
23use vortex_error::VortexResult;
24use vortex_error::vortex_bail;
25use vortex_session::VortexSession;
26use vortex_session::registry::ReadContext;
27use vortex_utils::aliases::hash_set::HashSet;
28
29use crate::IntoLayout;
30use crate::LayoutRef;
31use crate::LayoutStrategy;
32use crate::layouts::flat::FlatLayout;
33use crate::layouts::flat::flat_layout_inline_array_node;
34use crate::segments::SegmentSinkRef;
35use crate::sequence::SendableSequentialStream;
36use crate::sequence::SequencePointer;
37
38#[derive(Clone)]
39pub struct FlatLayoutStrategy {
40    /// Whether to include padding for memory-mapped reads.
41    pub include_padding: bool,
42    /// Maximum length of variable length statistics
43    pub max_variable_length_statistics_size: usize,
44    /// Optional set of allowed array encodings for normalization.
45    /// If None, then all are allowed.
46    pub allowed_encodings: Option<HashSet<ArrayId>>,
47}
48
49impl Default for FlatLayoutStrategy {
50    fn default() -> Self {
51        Self {
52            include_padding: true,
53            max_variable_length_statistics_size: 64,
54            allowed_encodings: None,
55        }
56    }
57}
58
59impl FlatLayoutStrategy {
60    /// Set whether to include padding for memory-mapped reads.
61    pub fn with_include_padding(mut self, include_padding: bool) -> Self {
62        self.include_padding = include_padding;
63        self
64    }
65
66    /// Set the maximum length of variable length statistics.
67    pub fn with_max_variable_length_statistics_size(mut self, size: usize) -> Self {
68        self.max_variable_length_statistics_size = size;
69        self
70    }
71
72    /// Set the allowed array encodings for normalization.
73    pub fn with_allow_encodings(mut self, allow_encodings: HashSet<ArrayId>) -> Self {
74        self.allowed_encodings = Some(allow_encodings);
75        self
76    }
77}
78
79fn truncate_scalar_stat<F: Fn(Scalar) -> Option<(Scalar, bool)>>(
80    statistics: StatsSetRef<'_>,
81    stat: Stat,
82    truncation: F,
83) {
84    if let Some(sv) = statistics.get(stat) {
85        if let Some((truncated_value, truncated)) = truncation(sv.into_inner()) {
86            if truncated && let Some(v) = truncated_value.into_value() {
87                statistics.set(stat, Precision::Inexact(v));
88            }
89        } else {
90            statistics.clear(stat)
91        }
92    }
93}
94
95#[async_trait]
96impl LayoutStrategy for FlatLayoutStrategy {
97    async fn write_stream(
98        &self,
99        ctx: ArrayContext,
100        segment_sink: SegmentSinkRef,
101        mut stream: SendableSequentialStream,
102        _eof: SequencePointer,
103        session: &VortexSession,
104    ) -> VortexResult<LayoutRef> {
105        let ctx = ctx.clone();
106        let Some(chunk) = stream.next().await else {
107            vortex_bail!("flat layout needs a single chunk");
108        };
109        let (sequence_id, chunk) = chunk?;
110
111        let row_count = chunk.len() as u64;
112
113        match chunk.dtype() {
114            DType::Utf8(n) => {
115                truncate_scalar_stat(chunk.statistics(), Stat::Min, |v| {
116                    lower_bound(
117                        BufferString::from_scalar(v)
118                            .vortex_expect("utf8 scalar must be a BufferString"),
119                        self.max_variable_length_statistics_size,
120                        *n,
121                    )
122                });
123                truncate_scalar_stat(chunk.statistics(), Stat::Max, |v| {
124                    upper_bound(
125                        BufferString::from_scalar(v)
126                            .vortex_expect("utf8 scalar must be a BufferString"),
127                        self.max_variable_length_statistics_size,
128                        *n,
129                    )
130                });
131            }
132            DType::Binary(n) => {
133                truncate_scalar_stat(chunk.statistics(), Stat::Min, |v| {
134                    lower_bound(
135                        ByteBuffer::from_scalar(v)
136                            .vortex_expect("binary scalar must be a ByteBuffer"),
137                        self.max_variable_length_statistics_size,
138                        *n,
139                    )
140                });
141                truncate_scalar_stat(chunk.statistics(), Stat::Max, |v| {
142                    upper_bound(
143                        ByteBuffer::from_scalar(v)
144                            .vortex_expect("binary scalar must be a ByteBuffer"),
145                        self.max_variable_length_statistics_size,
146                        *n,
147                    )
148                });
149            }
150            _ => {}
151        }
152
153        let chunk = if let Some(allowed) = &self.allowed_encodings {
154            chunk.normalize(&mut NormalizeOptions {
155                allowed,
156                operation: Operation::Error,
157            })?
158        } else {
159            chunk
160        };
161
162        let buffers = chunk.serialize(
163            &ctx,
164            session,
165            &SerializeOptions {
166                offset: 0,
167                include_padding: self.include_padding,
168            },
169        )?;
170        // there is at least the flatbuffer and the length
171        assert!(buffers.len() >= 2);
172        let array_node =
173            flat_layout_inline_array_node().then(|| buffers[buffers.len() - 2].clone());
174        let segment_id = segment_sink.write(sequence_id, buffers).await?;
175
176        let None = stream.next().await else {
177            vortex_bail!("flat layout received stream with more than a single chunk");
178        };
179        Ok(FlatLayout::new_with_metadata(
180            row_count,
181            stream.dtype().clone(),
182            segment_id,
183            ReadContext::new(ctx.to_ids()),
184            array_node,
185        )
186        .into_layout())
187    }
188
189    fn buffered_bytes(&self) -> u64 {
190        // FlatLayoutStrategy is a leaf strategy with no child strategies and no buffering
191        0
192    }
193}
194
195#[cfg(test)]
196mod tests {
197    use std::sync::Arc;
198
199    use vortex_array::ArrayContext;
200    use vortex_array::ArrayRef;
201    use vortex_array::IntoArray;
202    use vortex_array::LEGACY_SESSION;
203    use vortex_array::MaskFuture;
204    use vortex_array::ToCanonical;
205    use vortex_array::VortexSessionExecute;
206    use vortex_array::arrays::BoolArray;
207    use vortex_array::arrays::Dict;
208    use vortex_array::arrays::DictArray;
209    use vortex_array::arrays::PrimitiveArray;
210    use vortex_array::arrays::StructArray;
211    use vortex_array::arrays::struct_::StructArrayExt;
212    use vortex_array::builders::ArrayBuilder;
213    use vortex_array::builders::VarBinViewBuilder;
214    use vortex_array::dtype::DType;
215    use vortex_array::dtype::FieldName;
216    use vortex_array::dtype::FieldNames;
217    use vortex_array::dtype::Nullability;
218    use vortex_array::expr::root;
219    use vortex_array::expr::stats::Precision;
220    use vortex_array::expr::stats::Stat;
221    use vortex_array::expr::stats::StatsProviderExt;
222    use vortex_array::validity::Validity;
223    use vortex_array::vtable::VTable;
224    use vortex_buffer::BitBufferMut;
225    use vortex_buffer::buffer;
226    use vortex_error::VortexExpect;
227    use vortex_error::VortexResult;
228    use vortex_io::runtime::single::block_on;
229    use vortex_io::session::RuntimeSessionExt;
230    use vortex_mask::AllOr;
231    use vortex_mask::Mask;
232    use vortex_utils::aliases::hash_set::HashSet;
233
234    use crate::LayoutStrategy;
235    use crate::layouts::flat::writer::FlatLayoutStrategy;
236    use crate::segments::TestSegments;
237    use crate::sequence::SequenceId;
238    use crate::sequence::SequentialArrayStreamExt;
239    use crate::test::SESSION;
240
241    // Currently, flat layouts do not force compute stats during write, they only retain
242    // pre-computed stats.
243    #[should_panic]
244    #[test]
245    fn flat_stats() {
246        block_on(|handle| async {
247            let session = SESSION.clone().with_handle(handle);
248            let ctx = ArrayContext::empty();
249            let segments = Arc::new(TestSegments::default());
250            let (ptr, eof) = SequenceId::root().split();
251            let array = PrimitiveArray::new(buffer![1, 2, 3, 4, 5], Validity::AllValid);
252            let layout = FlatLayoutStrategy::default()
253                .write_stream(
254                    ctx,
255                    Arc::<TestSegments>::clone(&segments),
256                    array.into_array().to_array_stream().sequenced(ptr),
257                    eof,
258                    &session,
259                )
260                .await
261                .unwrap();
262
263            let result = layout
264                .new_reader("".into(), segments, &SESSION)
265                .unwrap()
266                .projection_evaluation(
267                    &(0..layout.row_count()),
268                    &root(),
269                    MaskFuture::new_true(layout.row_count().try_into().unwrap()),
270                )
271                .unwrap()
272                .await
273                .unwrap();
274
275            assert_eq!(
276                result.statistics().get_as::<bool>(Stat::IsSorted),
277                Some(Precision::Exact(true))
278            );
279        })
280    }
281
282    #[test]
283    fn truncates_variable_size_stats() {
284        block_on(|handle| async {
285            let session = SESSION.clone().with_handle(handle);
286            let ctx = ArrayContext::empty();
287            let segments = Arc::new(TestSegments::default());
288            let (ptr, eof) = SequenceId::root().split();
289            let mut builder =
290                VarBinViewBuilder::with_capacity(DType::Utf8(Nullability::NonNullable), 2);
291            builder.append_value("Long value to test that the statistics are actually truncated, it needs a bit of extra padding though");
292            builder.append_value("Another string that's meant to be smaller than the previous value, though still need extra padding");
293            let array = builder.finish();
294            let mut stats_ctx = session.create_execution_ctx();
295            array.statistics().set_iter(
296                array
297                    .statistics()
298                    .compute_all(&Stat::all().collect::<Vec<_>>(), &mut stats_ctx)
299                    .vortex_expect("stats computation should succeed for test array")
300                    .into_iter(),
301            );
302
303            let layout = FlatLayoutStrategy::default()
304                .write_stream(
305                    ctx,
306                    Arc::<TestSegments>::clone(&segments),
307                    array.into_array().to_array_stream().sequenced(ptr),
308                    eof,
309                    &session,
310                )
311                .await
312                .unwrap();
313
314            let result = layout
315                .new_reader("".into(), segments, &SESSION)
316                .unwrap()
317                .projection_evaluation(
318                    &(0..layout.row_count()),
319                    &root(),
320                    MaskFuture::new_true(layout.row_count().try_into().unwrap()),
321                )
322                .unwrap()
323                .await
324                .unwrap();
325
326            assert_eq!(
327                result.statistics().get_as::<String>(Stat::Min),
328                // The typo is correct, we need this to be truncated.
329                Some(Precision::Inexact(
330                    // spellchecker:ignore-next-line
331                    "Another string that's meant to be smaller than the previous valu".to_string()
332                ))
333            );
334            assert_eq!(
335                result.statistics().get_as::<String>(Stat::Max),
336                Some(Precision::Inexact(
337                    "Long value to test that the statistics are actually truncated, j".to_string()
338                ))
339            );
340        })
341    }
342
343    #[test]
344    fn struct_array_round_trip() {
345        block_on(|handle| async {
346            let session = SESSION.clone().with_handle(handle);
347            let mut validity_builder = BitBufferMut::with_capacity(2);
348            validity_builder.append(true);
349            validity_builder.append(false);
350            let validity_boolean_buffer = validity_builder.freeze();
351            let validity = Validity::Array(
352                BoolArray::new(validity_boolean_buffer.clone(), Validity::NonNullable).into_array(),
353            );
354            let array = StructArray::try_new(
355                FieldNames::from([FieldName::from("a"), FieldName::from("b")]),
356                vec![
357                    buffer![1_u64, 2].into_array(),
358                    buffer![3_u64, 4].into_array(),
359                ],
360                2,
361                validity,
362            )
363            .unwrap();
364
365            let ctx = ArrayContext::empty();
366
367            // Write the array into a byte buffer.
368            let (layout, segments) = {
369                let segments = Arc::new(TestSegments::default());
370                let (ptr, eof) = SequenceId::root().split();
371                let layout = FlatLayoutStrategy::default()
372                    .write_stream(
373                        ctx,
374                        Arc::<TestSegments>::clone(&segments),
375                        array.into_array().to_array_stream().sequenced(ptr),
376                        eof,
377                        &session,
378                    )
379                    .await
380                    .unwrap();
381
382                (layout, segments)
383            };
384
385            // We should be able to read the array we just wrote.
386            let result: ArrayRef = layout
387                .new_reader("".into(), segments, &SESSION)
388                .unwrap()
389                .projection_evaluation(
390                    &(0..layout.row_count()),
391                    &root(),
392                    MaskFuture::new_true(layout.row_count().try_into().unwrap()),
393                )
394                .unwrap()
395                .await
396                .unwrap();
397
398            assert_eq!(
399                result
400                    .validity()
401                    .unwrap()
402                    .to_mask(result.len(), &mut LEGACY_SESSION.create_execution_ctx())
403                    .unwrap()
404                    .bit_buffer(),
405                AllOr::Some(&validity_boolean_buffer)
406            );
407            assert_eq!(
408                result
409                    .to_struct()
410                    .unmasked_field_by_name("a")
411                    .unwrap()
412                    .to_primitive()
413                    .as_slice::<u64>(),
414                &[1, 2]
415            );
416            assert_eq!(
417                result
418                    .to_struct()
419                    .unmasked_field_by_name("b")
420                    .unwrap()
421                    .to_primitive()
422                    .as_slice::<u64>(),
423                &[3, 4]
424            );
425        })
426    }
427
428    #[test]
429    fn flat_invalid_array_fails() -> VortexResult<()> {
430        block_on(|handle| async {
431            let session = SESSION.clone().with_handle(handle);
432            let prim: PrimitiveArray = (0..10).collect();
433            let filter = prim.filter(Mask::from_indices(10, vec![2, 3]))?;
434
435            let ctx = ArrayContext::empty();
436
437            // Write the array into a byte buffer.
438            let (layout, _segments) = {
439                let segments = Arc::new(TestSegments::default());
440                let (ptr, eof) = SequenceId::root().split();
441                // Disallow all encodings so filter arrays fail normalization immediately.
442                let allowed = HashSet::default();
443                let layout = FlatLayoutStrategy::default()
444                    .with_allow_encodings(allowed)
445                    .write_stream(
446                        ctx,
447                        Arc::<TestSegments>::clone(&segments),
448                        filter.into_array().to_array_stream().sequenced(ptr),
449                        eof,
450                        &session,
451                    )
452                    .await;
453
454                (layout, segments)
455            };
456
457            let err = layout.expect_err("expected error");
458            assert!(
459                err.to_string()
460                    .contains("normalize forbids encoding (vortex.filter)"),
461                "unexpected error: {err}"
462            );
463
464            Ok(())
465        })
466    }
467
468    #[test]
469    fn flat_valid_array_writes() -> VortexResult<()> {
470        block_on(|handle| async {
471            let session = SESSION.clone().with_handle(handle);
472            let codes: PrimitiveArray = (0u32..10).collect();
473            let values: PrimitiveArray = (0..10).collect();
474            let dict = DictArray::new(codes.into_array(), values.into_array());
475
476            let ctx = ArrayContext::empty();
477
478            // Write the array into a byte buffer.
479            let (layout, _segments) = {
480                let segments = Arc::new(TestSegments::default());
481                let (ptr, eof) = SequenceId::root().split();
482                // Only allow the dict encoding; canonical primitive children remain permitted.
483                let mut allowed = HashSet::default();
484                allowed.insert(Dict.id());
485                let layout = FlatLayoutStrategy::default()
486                    .with_allow_encodings(allowed)
487                    .write_stream(
488                        ctx,
489                        Arc::<TestSegments>::clone(&segments),
490                        dict.into_array().to_array_stream().sequenced(ptr),
491                        eof,
492                        &session,
493                    )
494                    .await;
495
496                (layout, segments)
497            };
498
499            assert!(layout.is_ok());
500
501            Ok(())
502        })
503    }
504}