vortex_layout/layouts/flat/
writer.rs1use async_trait::async_trait;
5use futures::StreamExt;
6use vortex_array::ArrayContext;
7use vortex_array::DynArray;
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::session::ArrayRegistry;
20use vortex_array::stats::StatsSetRef;
21use vortex_buffer::BufferString;
22use vortex_buffer::ByteBuffer;
23use vortex_error::VortexExpect;
24use vortex_error::VortexResult;
25use vortex_error::vortex_bail;
26use vortex_io::runtime::Handle;
27use vortex_session::registry::ReadContext;
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 pub include_padding: bool,
42 pub max_variable_length_statistics_size: usize,
44 pub allowed_encodings: Option<ArrayRegistry>,
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 pub fn with_include_padding(mut self, include_padding: bool) -> Self {
62 self.include_padding = include_padding;
63 self
64 }
65
66 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 pub fn with_allow_encodings(mut self, allow_encodings: ArrayRegistry) -> 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 _handle: Handle,
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 &SerializeOptions {
165 offset: 0,
166 include_padding: self.include_padding,
167 },
168 )?;
169 assert!(buffers.len() >= 2);
171 let array_node =
172 flat_layout_inline_array_node().then(|| buffers[buffers.len() - 2].clone());
173 let segment_id = segment_sink.write(sequence_id, buffers).await?;
174
175 let None = stream.next().await else {
176 vortex_bail!("flat layout received stream with more than a single chunk");
177 };
178 Ok(FlatLayout::new_with_metadata(
179 row_count,
180 stream.dtype().clone(),
181 segment_id,
182 ReadContext::new(ctx.to_ids()),
183 array_node,
184 )
185 .into_layout())
186 }
187
188 fn buffered_bytes(&self) -> u64 {
189 0
191 }
192}
193
194#[cfg(test)]
195mod tests {
196 use std::sync::Arc;
197
198 use vortex_array::ArrayContext;
199 use vortex_array::ArrayRef;
200 use vortex_array::DynArray;
201 use vortex_array::IntoArray;
202 use vortex_array::MaskFuture;
203 use vortex_array::ToCanonical;
204 use vortex_array::arrays::BoolArray;
205 use vortex_array::arrays::Dict;
206 use vortex_array::arrays::DictArray;
207 use vortex_array::arrays::Primitive;
208 use vortex_array::arrays::PrimitiveArray;
209 use vortex_array::arrays::StructArray;
210 use vortex_array::builders::ArrayBuilder;
211 use vortex_array::builders::VarBinViewBuilder;
212 use vortex_array::dtype::DType;
213 use vortex_array::dtype::FieldName;
214 use vortex_array::dtype::FieldNames;
215 use vortex_array::dtype::Nullability;
216 use vortex_array::expr::root;
217 use vortex_array::expr::stats::Precision;
218 use vortex_array::expr::stats::Stat;
219 use vortex_array::expr::stats::StatsProviderExt;
220 use vortex_array::session::ArrayRegistry;
221 use vortex_array::validity::Validity;
222 use vortex_buffer::BitBufferMut;
223 use vortex_buffer::buffer;
224 use vortex_error::VortexExpect;
225 use vortex_error::VortexResult;
226 use vortex_io::runtime::single::block_on;
227 use vortex_mask::AllOr;
228 use vortex_mask::Mask;
229
230 use crate::LayoutStrategy;
231 use crate::layouts::flat::writer::FlatLayoutStrategy;
232 use crate::segments::TestSegments;
233 use crate::sequence::SequenceId;
234 use crate::sequence::SequentialArrayStreamExt;
235 use crate::test::SESSION;
236
237 #[should_panic]
240 #[test]
241 fn flat_stats() {
242 block_on(|handle| async {
243 let ctx = ArrayContext::empty();
244 let segments = Arc::new(TestSegments::default());
245 let (ptr, eof) = SequenceId::root().split();
246 let array = PrimitiveArray::new(buffer![1, 2, 3, 4, 5], Validity::AllValid);
247 let layout = FlatLayoutStrategy::default()
248 .write_stream(
249 ctx,
250 segments.clone(),
251 array.to_array_stream().sequenced(ptr),
252 eof,
253 handle,
254 )
255 .await
256 .unwrap();
257
258 let result = layout
259 .new_reader("".into(), segments, &SESSION)
260 .unwrap()
261 .projection_evaluation(
262 &(0..layout.row_count()),
263 &root(),
264 MaskFuture::new_true(layout.row_count().try_into().unwrap()),
265 )
266 .unwrap()
267 .await
268 .unwrap();
269
270 assert_eq!(
271 result.statistics().get_as::<bool>(Stat::IsSorted),
272 Some(Precision::Exact(true))
273 );
274 })
275 }
276
277 #[test]
278 fn truncates_variable_size_stats() {
279 block_on(|handle| async {
280 let ctx = ArrayContext::empty();
281 let segments = Arc::new(TestSegments::default());
282 let (ptr, eof) = SequenceId::root().split();
283 let mut builder =
284 VarBinViewBuilder::with_capacity(DType::Utf8(Nullability::NonNullable), 2);
285 builder.append_value("Long value to test that the statistics are actually truncated, it needs a bit of extra padding though");
286 builder.append_value("Another string that's meant to be smaller than the previous value, though still need extra padding");
287 let array = builder.finish();
288 array.statistics().set_iter(
289 array
290 .statistics()
291 .compute_all(&Stat::all().collect::<Vec<_>>())
292 .vortex_expect("stats computation should succeed for test array")
293 .into_iter(),
294 );
295
296 let layout = FlatLayoutStrategy::default()
297 .write_stream(
298 ctx,
299 segments.clone(),
300 array.to_array_stream().sequenced(ptr),
301 eof,
302 handle,
303 )
304 .await
305 .unwrap();
306
307 let result = layout
308 .new_reader("".into(), segments, &SESSION)
309 .unwrap()
310 .projection_evaluation(
311 &(0..layout.row_count()),
312 &root(),
313 MaskFuture::new_true(layout.row_count().try_into().unwrap()),
314 )
315 .unwrap()
316 .await
317 .unwrap();
318
319 assert_eq!(
320 result.statistics().get_as::<String>(Stat::Min),
321 Some(Precision::Inexact(
323 "Another string that's meant to be smaller than the previous valu".to_string()
325 ))
326 );
327 assert_eq!(
328 result.statistics().get_as::<String>(Stat::Max),
329 Some(Precision::Inexact(
330 "Long value to test that the statistics are actually truncated, j".to_string()
331 ))
332 );
333 })
334 }
335
336 #[test]
337 fn struct_array_round_trip() {
338 block_on(|handle| async {
339 let mut validity_builder = BitBufferMut::with_capacity(2);
340 validity_builder.append(true);
341 validity_builder.append(false);
342 let validity_boolean_buffer = validity_builder.freeze();
343 let validity = Validity::Array(
344 BoolArray::new(validity_boolean_buffer.clone(), Validity::NonNullable).into_array(),
345 );
346 let array = StructArray::try_new(
347 FieldNames::from([FieldName::from("a"), FieldName::from("b")]),
348 vec![
349 buffer![1_u64, 2].into_array(),
350 buffer![3_u64, 4].into_array(),
351 ],
352 2,
353 validity,
354 )
355 .unwrap();
356
357 let ctx = ArrayContext::empty();
358
359 let (layout, segments) = {
361 let segments = Arc::new(TestSegments::default());
362 let (ptr, eof) = SequenceId::root().split();
363 let layout = FlatLayoutStrategy::default()
364 .write_stream(
365 ctx,
366 segments.clone(),
367 array.to_array_stream().sequenced(ptr),
368 eof,
369 handle,
370 )
371 .await
372 .unwrap();
373
374 (layout, segments)
375 };
376
377 let result: ArrayRef = layout
379 .new_reader("".into(), segments, &SESSION)
380 .unwrap()
381 .projection_evaluation(
382 &(0..layout.row_count()),
383 &root(),
384 MaskFuture::new_true(layout.row_count().try_into().unwrap()),
385 )
386 .unwrap()
387 .await
388 .unwrap();
389
390 assert_eq!(
391 result.validity_mask().unwrap().bit_buffer(),
392 AllOr::Some(&validity_boolean_buffer)
393 );
394 assert_eq!(
395 result
396 .to_struct()
397 .unmasked_field_by_name("a")
398 .unwrap()
399 .to_primitive()
400 .as_slice::<u64>(),
401 &[1, 2]
402 );
403 assert_eq!(
404 result
405 .to_struct()
406 .unmasked_field_by_name("b")
407 .unwrap()
408 .to_primitive()
409 .as_slice::<u64>(),
410 &[3, 4]
411 );
412 })
413 }
414
415 #[test]
416 fn flat_invalid_array_fails() -> VortexResult<()> {
417 block_on(|handle| async {
418 let prim: PrimitiveArray = (0..10).collect();
419 let filter = prim.filter(Mask::from_indices(10, vec![2, 3]))?;
420
421 let ctx = ArrayContext::empty();
422
423 let (layout, _segments) = {
425 let segments = Arc::new(TestSegments::default());
426 let (ptr, eof) = SequenceId::root().split();
427 let allowed = ArrayRegistry::default();
429 allowed.register(Primitive::ID, Primitive);
430 let layout = FlatLayoutStrategy::default()
431 .with_allow_encodings(allowed)
432 .write_stream(
433 ctx,
434 segments.clone(),
435 filter.to_array_stream().sequenced(ptr),
436 eof,
437 handle,
438 )
439 .await;
440
441 (layout, segments)
442 };
443
444 let err = layout.expect_err("expected error");
445 assert!(
446 err.to_string()
447 .contains("normalize forbids encoding (vortex.filter)"),
448 "unexpected error: {err}"
449 );
450
451 Ok(())
452 })
453 }
454
455 #[test]
456 fn flat_valid_array_writes() -> VortexResult<()> {
457 block_on(|handle| async {
458 let codes: PrimitiveArray = (0u32..10).collect();
459 let values: PrimitiveArray = (0..10).collect();
460 let dict = DictArray::new(codes.into_array(), values.into_array());
461
462 let ctx = ArrayContext::empty();
463
464 let (layout, _segments) = {
466 let segments = Arc::new(TestSegments::default());
467 let (ptr, eof) = SequenceId::root().split();
468 let allowed = ArrayRegistry::default();
470 allowed.register(Primitive::ID, Primitive);
471 allowed.register(Dict::ID, Dict);
472 let layout = FlatLayoutStrategy::default()
473 .with_allow_encodings(allowed)
474 .write_stream(
475 ctx,
476 segments.clone(),
477 dict.to_array_stream().sequenced(ptr),
478 eof,
479 handle,
480 )
481 .await;
482
483 (layout, segments)
484 };
485
486 assert!(layout.is_ok());
487
488 Ok(())
489 })
490 }
491}