vortex_layout/layouts/stats/
stats_table.rs1use std::sync::Arc;
2
3use itertools::Itertools;
4use vortex_array::arrays::StructArray;
5use vortex_array::builders::{ArrayBuilder, ArrayBuilderExt, builder_with_capacity};
6use vortex_array::compute::sum;
7use vortex_array::stats::{Precision, Stat, StatsSet};
8use vortex_array::validity::Validity;
9use vortex_array::{Array, ArrayRef};
10use vortex_dtype::{DType, Nullability, PType, StructDType};
11use vortex_error::{VortexExpect, VortexResult, vortex_bail};
12
13#[derive(Clone)]
18pub struct StatsTable {
19 array: StructArray,
21 stats: Arc<[Stat]>,
23}
24
25impl StatsTable {
26 pub fn try_new(
29 column_dtype: DType,
30 array: StructArray,
31 stats: Arc<[Stat]>,
32 ) -> VortexResult<Self> {
33 if &Self::dtype_for_stats_table(&column_dtype, &stats) != array.dtype() {
34 vortex_bail!("Array dtype does not match expected stats table dtype");
35 }
36 Ok(Self::unchecked_new(array, stats))
37 }
38
39 pub fn unchecked_new(array: StructArray, stats: Arc<[Stat]>) -> Self {
41 Self { array, stats }
42 }
43
44 pub fn dtype_for_stats_table(column_dtype: &DType, present_stats: &[Stat]) -> DType {
46 assert!(present_stats.is_sorted(), "Stats must be sorted");
47 DType::Struct(
48 Arc::new(StructDType::from_iter(present_stats.iter().filter_map(
49 |stat| {
50 stat.dtype(column_dtype)
51 .map(|dtype| (stat.name(), dtype.as_nullable()))
52 },
53 ))),
54 Nullability::NonNullable,
55 )
56 }
57
58 pub fn array(&self) -> &StructArray {
60 &self.array
61 }
62
63 pub fn present_stats(&self) -> &[Stat] {
65 &self.stats
66 }
67
68 pub fn to_stats_set(&self, stats: &[Stat]) -> VortexResult<StatsSet> {
70 let mut stats_set = StatsSet::default();
71 for stat in stats {
72 let Some(array) = self.get_stat(*stat)? else {
73 continue;
74 };
75
76 match stat {
78 Stat::Min | Stat::Max | Stat::Sum => {
80 if let Some(s) = array.statistics().compute_stat(*stat)? {
81 stats_set.set(*stat, Precision::exact(s))
82 }
83 }
84 Stat::NullCount | Stat::NaNCount | Stat::UncompressedSizeInBytes => {
86 let sum = sum(&array)?
87 .cast(&DType::Primitive(PType::U64, Nullability::Nullable))?
88 .into_value();
89 stats_set.set(*stat, Precision::exact(sum));
90 }
91 Stat::IsConstant | Stat::IsSorted | Stat::IsStrictSorted => {}
93 }
94 }
95 Ok(stats_set)
96 }
97
98 pub fn get_stat(&self, stat: Stat) -> VortexResult<Option<ArrayRef>> {
100 Ok(self.array.field_by_name_opt(stat.name()).cloned())
101 }
102}
103
104pub struct StatsAccumulator {
111 stats: Arc<[Stat]>,
112 builders: Vec<Box<dyn ArrayBuilder>>,
113 length: usize,
114}
115
116impl StatsAccumulator {
117 pub fn new(dtype: DType, stats: &[Stat]) -> Self {
118 let (stats, builders): (Vec<Stat>, _) = stats
119 .iter()
120 .filter_map(|s| {
121 s.dtype(&dtype)
122 .map(|stat_dtype| (*s, builder_with_capacity(&stat_dtype.as_nullable(), 1024)))
123 })
124 .unzip();
125
126 Self {
127 stats: stats.into(),
128 builders,
129 length: 0,
130 }
131 }
132
133 pub fn stats(&self) -> &[Stat] {
134 &self.stats
135 }
136
137 pub fn push_chunk(&mut self, array: &dyn Array) -> VortexResult<()> {
138 for (s, builder) in self.stats.iter().zip_eq(self.builders.iter_mut()) {
139 if let Some(v) = array.statistics().compute_stat(*s)? {
140 builder.append_scalar_value(v)?;
141 } else {
142 builder.append_null();
143 }
144 }
145 self.length += 1;
146 Ok(())
147 }
148
149 pub fn as_stats_table(&mut self) -> Option<StatsTable> {
154 let mut names = Vec::new();
155 let mut fields = Vec::new();
156 let mut stats = Vec::new();
157
158 for (stat, builder) in self
159 .stats
160 .iter()
161 .zip(self.builders.iter_mut())
162 .sorted_unstable_by_key(|&(&s, _)| s)
164 {
165 let values = builder.finish();
166
167 if values
169 .invalid_count()
170 .vortex_expect("failed to get invalid count")
171 == values.len()
172 {
173 continue;
174 }
175
176 stats.push(*stat);
177 names.push(stat.to_string().into());
178 fields.push(values);
179 }
180
181 if names.is_empty() {
182 return None;
183 }
184
185 Some(StatsTable {
186 array: StructArray::try_new(names.into(), fields, self.length, Validity::NonNullable)
187 .vortex_expect("Failed to create stats table"),
188 stats: stats.into(),
189 })
190 }
191}