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::serde::SerializeOptions;
7use vortex_array::stats::{Precision, Stat, StatsProvider};
8use vortex_array::{Array, ArrayContext};
9use vortex_dtype::DType;
10use vortex_error::{VortexResult, vortex_bail};
11use vortex_io::runtime::Handle;
12
13use crate::layouts::flat::FlatLayout;
14use crate::layouts::zoned::{lower_bound, upper_bound};
15use crate::segments::SegmentSinkRef;
16use crate::sequence::{SendableSequentialStream, SequencePointer};
17use crate::{IntoLayout, LayoutRef, LayoutStrategy};
18
19#[derive(Clone)]
20pub struct FlatLayoutStrategy {
21    /// Whether to include padding for memory-mapped reads.
22    pub include_padding: bool,
23    /// Maximum length of variable length statistics
24    pub max_variable_length_statistics_size: usize,
25}
26
27impl Default for FlatLayoutStrategy {
28    fn default() -> Self {
29        Self {
30            include_padding: true,
31            max_variable_length_statistics_size: 64,
32        }
33    }
34}
35
36#[async_trait]
37impl LayoutStrategy for FlatLayoutStrategy {
38    async fn write_stream(
39        &self,
40        ctx: ArrayContext,
41        segment_sink: SegmentSinkRef,
42        mut stream: SendableSequentialStream,
43        _eof: SequencePointer,
44        _handle: Handle,
45    ) -> VortexResult<LayoutRef> {
46        let ctx = ctx.clone();
47        let options = self.clone();
48        let Some(chunk) = stream.next().await else {
49            vortex_bail!("flat layout needs a single chunk");
50        };
51        let (sequence_id, chunk) = chunk?;
52
53        let row_count = chunk.len() as u64;
54
55        match chunk.dtype() {
56            DType::Utf8(_) => {
57                if let Some(sv) = chunk.statistics().get(Stat::Min) {
58                    let (value, truncated) = lower_bound(
59                        sv.into_inner().as_utf8(),
60                        options.max_variable_length_statistics_size,
61                    );
62                    if truncated {
63                        chunk
64                            .statistics()
65                            .set(Stat::Min, Precision::Inexact(value.into_value()));
66                    }
67                }
68
69                if let Some(sv) = chunk.statistics().get(Stat::Max) {
70                    let (value, truncated) = upper_bound(
71                        sv.into_inner().as_utf8(),
72                        options.max_variable_length_statistics_size,
73                    );
74                    if let Some(upper_bound) = value {
75                        if truncated {
76                            chunk
77                                .statistics()
78                                .set(Stat::Max, Precision::Inexact(upper_bound.into_value()));
79                        }
80                    } else {
81                        chunk.statistics().clear(Stat::Max)
82                    }
83                }
84            }
85            DType::Binary(_) => {
86                if let Some(sv) = chunk.statistics().get(Stat::Min) {
87                    let (value, truncated) = lower_bound(
88                        sv.into_inner().as_binary(),
89                        options.max_variable_length_statistics_size,
90                    );
91                    if truncated {
92                        chunk
93                            .statistics()
94                            .set(Stat::Min, Precision::Inexact(value.into_value()));
95                    }
96                }
97
98                if let Some(sv) = chunk.statistics().get(Stat::Max) {
99                    let (value, truncated) = upper_bound(
100                        sv.into_inner().as_binary(),
101                        options.max_variable_length_statistics_size,
102                    );
103                    if let Some(upper_bound) = value {
104                        if truncated {
105                            chunk
106                                .statistics()
107                                .set(Stat::Max, Precision::Inexact(upper_bound.into_value()));
108                        }
109                    } else {
110                        chunk.statistics().clear(Stat::Max)
111                    }
112                }
113            }
114            _ => {}
115        }
116
117        // TODO(os): spawn serialization
118        let buffers = chunk.serialize(
119            &ctx,
120            &SerializeOptions {
121                offset: 0,
122                include_padding: options.include_padding,
123            },
124        )?;
125        let segment_id = segment_sink.write(sequence_id, buffers).await?;
126
127        let None = stream.next().await else {
128            vortex_bail!("flat layout received stream with more than a single chunk");
129        };
130        Ok(
131            FlatLayout::new(row_count, stream.dtype().clone(), segment_id, ctx.clone())
132                .into_layout(),
133        )
134    }
135
136    fn buffered_bytes(&self) -> u64 {
137        // FlatLayoutStrategy is a leaf strategy with no child strategies and no buffering
138        0
139    }
140}
141
142#[cfg(test)]
143mod tests {
144    use std::sync::Arc;
145
146    use arrow_buffer::BooleanBufferBuilder;
147    use vortex_array::arrays::{BoolArray, PrimitiveArray, StructArray};
148    use vortex_array::builders::{ArrayBuilder, VarBinViewBuilder};
149    use vortex_array::stats::{Precision, Stat, StatsProviderExt};
150    use vortex_array::validity::Validity;
151    use vortex_array::{Array, ArrayContext, ArrayRef, IntoArray, MaskFuture, ToCanonical};
152    use vortex_buffer::buffer;
153    use vortex_dtype::{DType, FieldName, FieldNames, Nullability};
154    use vortex_error::VortexUnwrap;
155    use vortex_expr::root;
156    use vortex_io::runtime::single::block_on;
157    use vortex_mask::AllOr;
158
159    use crate::LayoutStrategy;
160    use crate::layouts::flat::writer::FlatLayoutStrategy;
161    use crate::segments::TestSegments;
162    use crate::sequence::{SequenceId, SequentialArrayStreamExt};
163
164    // Currently, flat layouts do not force compute stats during write, they only retain
165    // pre-computed stats.
166    #[should_panic]
167    #[test]
168    fn flat_stats() {
169        block_on(|handle| async {
170            let ctx = ArrayContext::empty();
171            let segments = Arc::new(TestSegments::default());
172            let (ptr, eof) = SequenceId::root().split();
173            let array = PrimitiveArray::new(buffer![1, 2, 3, 4, 5], Validity::AllValid);
174            let layout = FlatLayoutStrategy::default()
175                .write_stream(
176                    ctx,
177                    segments.clone(),
178                    array.to_array_stream().sequenced(ptr),
179                    eof,
180                    handle,
181                )
182                .await
183                .unwrap();
184
185            let result = layout
186                .new_reader("".into(), segments)
187                .unwrap()
188                .projection_evaluation(
189                    &(0..layout.row_count()),
190                    &root(),
191                    MaskFuture::new_true(layout.row_count().try_into().unwrap()),
192                )
193                .unwrap()
194                .await
195                .unwrap();
196
197            assert_eq!(
198                result.statistics().get_as::<bool>(Stat::IsSorted),
199                Some(Precision::Exact(true))
200            );
201        })
202    }
203
204    #[test]
205    fn truncates_variable_size_stats() {
206        block_on(|handle| async {
207            let ctx = ArrayContext::empty();
208            let segments = Arc::new(TestSegments::default());
209            let (ptr, eof) = SequenceId::root().split();
210            let mut builder =
211                VarBinViewBuilder::with_capacity(DType::Utf8(Nullability::NonNullable), 2);
212            builder.append_value("Long value to test that the statistics are actually truncated, it needs a bit of extra padding though");
213            builder.append_value("Another string that's meant to be smaller than the previous value, though still need extra padding");
214            let array = builder.finish();
215            array.statistics().set_iter(
216                array
217                    .statistics()
218                    .compute_all(&Stat::all().collect::<Vec<_>>())
219                    .vortex_unwrap()
220                    .into_iter(),
221            );
222
223            let layout = FlatLayoutStrategy::default()
224                .write_stream(
225                    ctx,
226                    segments.clone(),
227                    array.to_array_stream().sequenced(ptr),
228                    eof,
229                    handle,
230                )
231                .await
232                .unwrap();
233
234            let result = layout
235                .new_reader("".into(), segments)
236                .unwrap()
237                .projection_evaluation(
238                    &(0..layout.row_count()),
239                    &root(),
240                    MaskFuture::new_true(layout.row_count().try_into().unwrap()),
241                )
242                .unwrap()
243                .await
244                .unwrap();
245
246            assert_eq!(
247                result.statistics().get_as::<String>(Stat::Min),
248                // The typo is correct, we need this to be truncated.
249                Some(Precision::Inexact(
250                    // spellchecker:ignore-next-line
251                    "Another string that's meant to be smaller than the previous valu".to_string()
252                ))
253            );
254            assert_eq!(
255                result.statistics().get_as::<String>(Stat::Max),
256                Some(Precision::Inexact(
257                    "Long value to test that the statistics are actually truncated, j".to_string()
258                ))
259            );
260        })
261    }
262
263    #[test]
264    fn struct_array_round_trip() {
265        block_on(|handle| async {
266            let mut validity_builder = BooleanBufferBuilder::new(2);
267            validity_builder.append(true);
268            validity_builder.append(false);
269            let validity_boolean_buffer = validity_builder.finish();
270            let validity = Validity::Array(
271                BoolArray::from_bool_buffer(validity_boolean_buffer.clone(), Validity::NonNullable)
272                    .into_array(),
273            );
274            let array = StructArray::try_new(
275                FieldNames::from([FieldName::from("a"), FieldName::from("b")]),
276                vec![
277                    buffer![1_u64, 2].into_array(),
278                    buffer![3_u64, 4].into_array(),
279                ],
280                2,
281                validity,
282            )
283            .unwrap();
284
285            let ctx = ArrayContext::empty();
286
287            // Write the array into a byte buffer.
288            let (layout, segments) = {
289                let segments = Arc::new(TestSegments::default());
290                let (ptr, eof) = SequenceId::root().split();
291                let layout = FlatLayoutStrategy::default()
292                    .write_stream(
293                        ctx,
294                        segments.clone(),
295                        array.to_array_stream().sequenced(ptr),
296                        eof,
297                        handle,
298                    )
299                    .await
300                    .unwrap();
301
302                (layout, segments)
303            };
304
305            // We should be able to read the array we just wrote.
306            let result: ArrayRef = layout
307                .new_reader("".into(), segments)
308                .unwrap()
309                .projection_evaluation(
310                    &(0..layout.row_count()),
311                    &root(),
312                    MaskFuture::new_true(layout.row_count().try_into().unwrap()),
313                )
314                .unwrap()
315                .await
316                .unwrap();
317
318            assert_eq!(
319                result.validity_mask().boolean_buffer(),
320                AllOr::Some(&validity_boolean_buffer)
321            );
322            assert_eq!(
323                result
324                    .to_struct()
325                    .field_by_name("a")
326                    .unwrap()
327                    .to_primitive()
328                    .as_slice::<u64>(),
329                &[1, 2]
330            );
331            assert_eq!(
332                result
333                    .to_struct()
334                    .field_by_name("b")
335                    .unwrap()
336                    .to_primitive()
337                    .as_slice::<u64>(),
338                &[3, 4]
339            );
340        })
341    }
342}