vortex_layout/layouts/flat/
writer.rs1use 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 pub include_padding: bool,
42 pub max_variable_length_statistics_size: usize,
44 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 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: 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 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 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::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::PrimitiveArray;
208 use vortex_array::arrays::StructArray;
209 use vortex_array::arrays::struct_::StructArrayExt;
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::validity::Validity;
221 use vortex_array::vtable::VTable;
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_io::session::RuntimeSessionExt;
228 use vortex_mask::AllOr;
229 use vortex_mask::Mask;
230 use vortex_utils::aliases::hash_set::HashSet;
231
232 use crate::LayoutStrategy;
233 use crate::layouts::flat::writer::FlatLayoutStrategy;
234 use crate::segments::TestSegments;
235 use crate::sequence::SequenceId;
236 use crate::sequence::SequentialArrayStreamExt;
237 use crate::test::SESSION;
238
239 #[should_panic]
242 #[test]
243 fn flat_stats() {
244 block_on(|handle| async {
245 let session = SESSION.clone().with_handle(handle);
246 let ctx = ArrayContext::empty();
247 let segments = Arc::new(TestSegments::default());
248 let (ptr, eof) = SequenceId::root().split();
249 let array = PrimitiveArray::new(buffer![1, 2, 3, 4, 5], Validity::AllValid);
250 let layout = FlatLayoutStrategy::default()
251 .write_stream(
252 ctx,
253 Arc::<TestSegments>::clone(&segments),
254 array.into_array().to_array_stream().sequenced(ptr),
255 eof,
256 &session,
257 )
258 .await
259 .unwrap();
260
261 let result = layout
262 .new_reader("".into(), segments, &SESSION)
263 .unwrap()
264 .projection_evaluation(
265 &(0..layout.row_count()),
266 &root(),
267 MaskFuture::new_true(layout.row_count().try_into().unwrap()),
268 )
269 .unwrap()
270 .await
271 .unwrap();
272
273 assert_eq!(
274 result.statistics().get_as::<bool>(Stat::IsSorted),
275 Some(Precision::Exact(true))
276 );
277 })
278 }
279
280 #[test]
281 fn truncates_variable_size_stats() {
282 block_on(|handle| async {
283 let session = SESSION.clone().with_handle(handle);
284 let ctx = ArrayContext::empty();
285 let segments = Arc::new(TestSegments::default());
286 let (ptr, eof) = SequenceId::root().split();
287 let mut builder =
288 VarBinViewBuilder::with_capacity(DType::Utf8(Nullability::NonNullable), 2);
289 builder.append_value("Long value to test that the statistics are actually truncated, it needs a bit of extra padding though");
290 builder.append_value("Another string that's meant to be smaller than the previous value, though still need extra padding");
291 let array = builder.finish();
292 array.statistics().set_iter(
293 array
294 .statistics()
295 .compute_all(&Stat::all().collect::<Vec<_>>())
296 .vortex_expect("stats computation should succeed for test array")
297 .into_iter(),
298 );
299
300 let layout = FlatLayoutStrategy::default()
301 .write_stream(
302 ctx,
303 Arc::<TestSegments>::clone(&segments),
304 array.into_array().to_array_stream().sequenced(ptr),
305 eof,
306 &session,
307 )
308 .await
309 .unwrap();
310
311 let result = layout
312 .new_reader("".into(), segments, &SESSION)
313 .unwrap()
314 .projection_evaluation(
315 &(0..layout.row_count()),
316 &root(),
317 MaskFuture::new_true(layout.row_count().try_into().unwrap()),
318 )
319 .unwrap()
320 .await
321 .unwrap();
322
323 assert_eq!(
324 result.statistics().get_as::<String>(Stat::Min),
325 Some(Precision::Inexact(
327 "Another string that's meant to be smaller than the previous valu".to_string()
329 ))
330 );
331 assert_eq!(
332 result.statistics().get_as::<String>(Stat::Max),
333 Some(Precision::Inexact(
334 "Long value to test that the statistics are actually truncated, j".to_string()
335 ))
336 );
337 })
338 }
339
340 #[test]
341 fn struct_array_round_trip() {
342 block_on(|handle| async {
343 let session = SESSION.clone().with_handle(handle);
344 let mut validity_builder = BitBufferMut::with_capacity(2);
345 validity_builder.append(true);
346 validity_builder.append(false);
347 let validity_boolean_buffer = validity_builder.freeze();
348 let validity = Validity::Array(
349 BoolArray::new(validity_boolean_buffer.clone(), Validity::NonNullable).into_array(),
350 );
351 let array = StructArray::try_new(
352 FieldNames::from([FieldName::from("a"), FieldName::from("b")]),
353 vec![
354 buffer![1_u64, 2].into_array(),
355 buffer![3_u64, 4].into_array(),
356 ],
357 2,
358 validity,
359 )
360 .unwrap();
361
362 let ctx = ArrayContext::empty();
363
364 let (layout, segments) = {
366 let segments = Arc::new(TestSegments::default());
367 let (ptr, eof) = SequenceId::root().split();
368 let layout = FlatLayoutStrategy::default()
369 .write_stream(
370 ctx,
371 Arc::<TestSegments>::clone(&segments),
372 array.into_array().to_array_stream().sequenced(ptr),
373 eof,
374 &session,
375 )
376 .await
377 .unwrap();
378
379 (layout, segments)
380 };
381
382 let result: ArrayRef = layout
384 .new_reader("".into(), segments, &SESSION)
385 .unwrap()
386 .projection_evaluation(
387 &(0..layout.row_count()),
388 &root(),
389 MaskFuture::new_true(layout.row_count().try_into().unwrap()),
390 )
391 .unwrap()
392 .await
393 .unwrap();
394
395 assert_eq!(
396 result.validity_mask().unwrap().bit_buffer(),
397 AllOr::Some(&validity_boolean_buffer)
398 );
399 assert_eq!(
400 result
401 .to_struct()
402 .unmasked_field_by_name("a")
403 .unwrap()
404 .to_primitive()
405 .as_slice::<u64>(),
406 &[1, 2]
407 );
408 assert_eq!(
409 result
410 .to_struct()
411 .unmasked_field_by_name("b")
412 .unwrap()
413 .to_primitive()
414 .as_slice::<u64>(),
415 &[3, 4]
416 );
417 })
418 }
419
420 #[test]
421 fn flat_invalid_array_fails() -> VortexResult<()> {
422 block_on(|handle| async {
423 let session = SESSION.clone().with_handle(handle);
424 let prim: PrimitiveArray = (0..10).collect();
425 let filter = prim.filter(Mask::from_indices(10, vec![2, 3]))?;
426
427 let ctx = ArrayContext::empty();
428
429 let (layout, _segments) = {
431 let segments = Arc::new(TestSegments::default());
432 let (ptr, eof) = SequenceId::root().split();
433 let allowed = HashSet::default();
435 let layout = FlatLayoutStrategy::default()
436 .with_allow_encodings(allowed)
437 .write_stream(
438 ctx,
439 Arc::<TestSegments>::clone(&segments),
440 filter.into_array().to_array_stream().sequenced(ptr),
441 eof,
442 &session,
443 )
444 .await;
445
446 (layout, segments)
447 };
448
449 let err = layout.expect_err("expected error");
450 assert!(
451 err.to_string()
452 .contains("normalize forbids encoding (vortex.filter)"),
453 "unexpected error: {err}"
454 );
455
456 Ok(())
457 })
458 }
459
460 #[test]
461 fn flat_valid_array_writes() -> VortexResult<()> {
462 block_on(|handle| async {
463 let session = SESSION.clone().with_handle(handle);
464 let codes: PrimitiveArray = (0u32..10).collect();
465 let values: PrimitiveArray = (0..10).collect();
466 let dict = DictArray::new(codes.into_array(), values.into_array());
467
468 let ctx = ArrayContext::empty();
469
470 let (layout, _segments) = {
472 let segments = Arc::new(TestSegments::default());
473 let (ptr, eof) = SequenceId::root().split();
474 let mut allowed = HashSet::default();
476 allowed.insert(Dict.id());
477 let layout = FlatLayoutStrategy::default()
478 .with_allow_encodings(allowed)
479 .write_stream(
480 ctx,
481 Arc::<TestSegments>::clone(&segments),
482 dict.into_array().to_array_stream().sequenced(ptr),
483 eof,
484 &session,
485 )
486 .await;
487
488 (layout, segments)
489 };
490
491 assert!(layout.is_ok());
492
493 Ok(())
494 })
495 }
496}