vortex_layout/layouts/
file_stats.rs1use std::sync::Arc;
2
3use itertools::Itertools;
4use vortex_array::stats::{Stat, StatsSet};
5use vortex_array::{Array, ArrayRef, ToCanonical};
6use vortex_dtype::DType;
7use vortex_error::{VortexExpect, VortexResult};
8
9use crate::layouts::zoned::zone_map::StatsAccumulator;
10use crate::segments::SegmentWriter;
11use crate::{LayoutRef, LayoutWriter};
12
13pub struct FileStatsLayoutWriter {
17 inner: Box<dyn LayoutWriter>,
18 stats: Arc<[Stat]>,
19 stats_accumulators: Vec<StatsAccumulator>,
20}
21
22impl FileStatsLayoutWriter {
23 pub fn new(
24 inner: Box<dyn LayoutWriter>,
25 dtype: &DType,
26 stats: Arc<[Stat]>,
27 max_variable_length_statistics_size: usize,
28 ) -> VortexResult<Self> {
29 let stats_accumulators = match dtype.as_struct() {
30 Some(dtype) => dtype
31 .fields()
32 .map(|field_dtype| {
33 StatsAccumulator::new(&field_dtype, &stats, max_variable_length_statistics_size)
34 })
35 .collect(),
36 None => [StatsAccumulator::new(
37 dtype,
38 &stats,
39 max_variable_length_statistics_size,
40 )]
41 .into(),
42 };
43
44 Ok(Self {
45 inner,
46 stats,
47 stats_accumulators,
48 })
49 }
50
51 pub fn into_stats_sets(self) -> Vec<StatsSet> {
53 self.stats_accumulators
54 .into_iter()
55 .map(|mut acc| {
56 acc.as_stats_table()
57 .map(|table| {
58 table
59 .to_stats_set(&self.stats)
60 .vortex_expect("shouldn't fail to convert table we just created")
61 })
62 .unwrap_or_default()
63 })
64 .collect()
65 }
66}
67
68impl LayoutWriter for FileStatsLayoutWriter {
69 fn push_chunk(
70 &mut self,
71 segment_writer: &mut dyn SegmentWriter,
72 chunk: ArrayRef,
73 ) -> VortexResult<()> {
74 if chunk.dtype().is_struct() {
75 let chunk = chunk.to_struct()?;
76 for (acc, field) in self.stats_accumulators.iter_mut().zip_eq(chunk.fields()) {
77 acc.push_chunk(field)?;
78 }
79 } else {
80 self.stats_accumulators[0].push_chunk(&chunk)?;
81 }
82 self.inner.push_chunk(segment_writer, chunk)
83 }
84
85 fn flush(&mut self, segment_writer: &mut dyn SegmentWriter) -> VortexResult<()> {
86 self.inner.flush(segment_writer)
87 }
88
89 fn finish(&mut self, segment_writer: &mut dyn SegmentWriter) -> VortexResult<LayoutRef> {
90 self.inner.finish(segment_writer)
91 }
92}