1use std::sync::Arc;
24
25use crate::file_groups::FileGroup;
26use crate::PartitionedFile;
27
28use arrow::array::RecordBatch;
29use arrow::compute::SortColumn;
30use arrow::datatypes::SchemaRef;
31use arrow::row::{Row, Rows};
32use datafusion_common::stats::Precision;
33use datafusion_common::{
34 plan_datafusion_err, plan_err, DataFusionError, Result, ScalarValue,
35};
36use datafusion_physical_expr::expressions::Column;
37use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr};
38use datafusion_physical_plan::{ColumnStatistics, Statistics};
39
40use futures::{Stream, StreamExt};
41
42pub(crate) struct MinMaxStatistics {
46 min_by_sort_order: Rows,
47 max_by_sort_order: Rows,
48 sort_order: LexOrdering,
49}
50
51impl MinMaxStatistics {
52 #[allow(unused)]
54 pub fn sort_order(&self) -> &LexOrdering {
55 &self.sort_order
56 }
57
58 #[allow(unused)]
60 pub fn min(&self, idx: usize) -> Row {
61 self.min_by_sort_order.row(idx)
62 }
63
64 pub fn max(&self, idx: usize) -> Row {
66 self.max_by_sort_order.row(idx)
67 }
68
69 pub fn new_from_files<'a>(
70 projected_sort_order: &LexOrdering, projected_schema: &SchemaRef, projection: Option<&[usize]>, files: impl IntoIterator<Item = &'a PartitionedFile>,
74 ) -> Result<Self> {
75 let Some(statistics_and_partition_values) = files
76 .into_iter()
77 .map(|file| {
78 file.statistics
79 .as_ref()
80 .zip(Some(file.partition_values.as_slice()))
81 })
82 .collect::<Option<Vec<_>>>()
83 else {
84 return plan_err!("Parquet file missing statistics");
85 };
86
87 let get_min_max = |i: usize| -> Result<(Vec<ScalarValue>, Vec<ScalarValue>)> {
89 Ok(statistics_and_partition_values
90 .iter()
91 .map(|(s, pv)| {
92 if i < s.column_statistics.len() {
93 s.column_statistics[i]
94 .min_value
95 .get_value()
96 .cloned()
97 .zip(s.column_statistics[i].max_value.get_value().cloned())
98 .ok_or_else(|| plan_datafusion_err!("statistics not found"))
99 } else {
100 let partition_value = &pv[i - s.column_statistics.len()];
101 Ok((partition_value.clone(), partition_value.clone()))
102 }
103 })
104 .collect::<Result<Vec<_>>>()?
105 .into_iter()
106 .unzip())
107 };
108
109 let Some(sort_columns) =
110 sort_columns_from_physical_sort_exprs(projected_sort_order)
111 else {
112 return plan_err!("sort expression must be on column");
113 };
114
115 let min_max_schema = Arc::new(
117 projected_schema
118 .project(&(sort_columns.iter().map(|c| c.index()).collect::<Vec<_>>()))?,
119 );
120
121 let min_max_sort_order = projected_sort_order
122 .iter()
123 .zip(sort_columns.iter())
124 .enumerate()
125 .map(|(idx, (sort_expr, col))| {
126 let expr = Arc::new(Column::new(col.name(), idx));
127 PhysicalSortExpr::new(expr, sort_expr.options)
128 });
129 let min_max_sort_order = LexOrdering::new(min_max_sort_order).unwrap();
131
132 let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns
133 .iter()
134 .map(|c| {
135 let i = projection
139 .map(|p| p[c.index()])
140 .unwrap_or_else(|| c.index());
141
142 let (min, max) = get_min_max(i).map_err(|e| {
143 e.context(format!("get min/max for column: '{}'", c.name()))
144 })?;
145 Ok((
146 ScalarValue::iter_to_array(min)?,
147 ScalarValue::iter_to_array(max)?,
148 ))
149 })
150 .collect::<Result<Vec<_>>>()
151 .map_err(|e| e.context("collect min/max values"))?
152 .into_iter()
153 .unzip();
154
155 Self::new(
156 &min_max_sort_order,
157 &min_max_schema,
158 RecordBatch::try_new(Arc::clone(&min_max_schema), min_values).map_err(
159 |e| {
160 DataFusionError::ArrowError(
161 Box::new(e),
162 Some("\ncreate min batch".to_string()),
163 )
164 },
165 )?,
166 RecordBatch::try_new(Arc::clone(&min_max_schema), max_values).map_err(
167 |e| {
168 DataFusionError::ArrowError(
169 Box::new(e),
170 Some("\ncreate max batch".to_string()),
171 )
172 },
173 )?,
174 )
175 }
176
177 pub fn new(
178 sort_order: &LexOrdering,
179 schema: &SchemaRef,
180 min_values: RecordBatch,
181 max_values: RecordBatch,
182 ) -> Result<Self> {
183 use arrow::row::*;
184
185 let sort_fields = sort_order
186 .iter()
187 .map(|expr| {
188 expr.expr
189 .data_type(schema)
190 .map(|data_type| SortField::new_with_options(data_type, expr.options))
191 })
192 .collect::<Result<Vec<_>>>()
193 .map_err(|e| e.context("create sort fields"))?;
194 let converter = RowConverter::new(sort_fields)?;
195
196 let Some(sort_columns) = sort_columns_from_physical_sort_exprs(sort_order) else {
197 return plan_err!("sort expression must be on column");
198 };
199
200 let (new_min_cols, new_max_cols): (Vec<_>, Vec<_>) = sort_order
202 .iter()
203 .zip(sort_columns.iter().copied())
204 .map(|(sort_expr, column)| {
205 let maxes = max_values.column_by_name(column.name());
206 let mins = min_values.column_by_name(column.name());
207 let opt_value = if sort_expr.options.descending {
208 maxes.zip(mins)
209 } else {
210 mins.zip(maxes)
211 };
212 opt_value.ok_or_else(|| {
213 plan_datafusion_err!(
214 "missing column in MinMaxStatistics::new: '{}'",
215 column.name()
216 )
217 })
218 })
219 .collect::<Result<Vec<_>>>()?
220 .into_iter()
221 .unzip();
222
223 let [min, max] = [new_min_cols, new_max_cols].map(|cols| {
224 let values = RecordBatch::try_new(
225 min_values.schema(),
226 cols.into_iter().cloned().collect(),
227 )?;
228 let sorting_columns = sort_order
229 .iter()
230 .zip(sort_columns.iter().copied())
231 .map(|(sort_expr, column)| {
232 let schema = values.schema();
233 let idx = schema.index_of(column.name())?;
234
235 Ok(SortColumn {
236 values: Arc::clone(values.column(idx)),
237 options: Some(sort_expr.options),
238 })
239 })
240 .collect::<Result<Vec<_>>>()
241 .map_err(|e| e.context("create sorting columns"))?;
242 converter
243 .convert_columns(
244 &sorting_columns
245 .into_iter()
246 .map(|c| c.values)
247 .collect::<Vec<_>>(),
248 )
249 .map_err(|e| {
250 DataFusionError::ArrowError(
251 Box::new(e),
252 Some("convert columns".to_string()),
253 )
254 })
255 });
256
257 Ok(Self {
258 min_by_sort_order: min.map_err(|e| e.context("build min rows"))?,
259 max_by_sort_order: max.map_err(|e| e.context("build max rows"))?,
260 sort_order: sort_order.clone(),
261 })
262 }
263
264 pub fn min_values_sorted(&self) -> Vec<(usize, Row<'_>)> {
266 let mut sort: Vec<_> = self.min_by_sort_order.iter().enumerate().collect();
267 sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b));
268 sort
269 }
270
271 pub fn is_sorted(&self) -> bool {
273 self.max_by_sort_order
274 .iter()
275 .zip(self.min_by_sort_order.iter().skip(1))
276 .all(|(max, next_min)| max < next_min)
277 }
278}
279
280fn sort_columns_from_physical_sort_exprs(
281 sort_order: &LexOrdering,
282) -> Option<Vec<&Column>> {
283 sort_order
284 .iter()
285 .map(|expr| expr.expr.as_any().downcast_ref::<Column>())
286 .collect()
287}
288
289#[deprecated(
295 since = "47.0.0",
296 note = "Please use `get_files_with_limit` and `compute_all_files_statistics` instead"
297)]
298#[allow(unused)]
299pub async fn get_statistics_with_limit(
300 all_files: impl Stream<Item = Result<(PartitionedFile, Arc<Statistics>)>>,
301 file_schema: SchemaRef,
302 limit: Option<usize>,
303 collect_stats: bool,
304) -> Result<(FileGroup, Statistics)> {
305 let mut result_files = FileGroup::default();
306 let size = file_schema.fields().len();
313 let mut col_stats_set = vec![ColumnStatistics::default(); size];
314 let mut num_rows = Precision::<usize>::Absent;
315 let mut total_byte_size = Precision::<usize>::Absent;
316
317 let mut all_files = Box::pin(all_files.fuse());
319
320 if let Some(first_file) = all_files.next().await {
321 let (mut file, file_stats) = first_file?;
322 file.statistics = Some(Arc::clone(&file_stats));
323 result_files.push(file);
324
325 num_rows = file_stats.num_rows;
327 total_byte_size = file_stats.total_byte_size;
328 for (index, file_column) in
329 file_stats.column_statistics.clone().into_iter().enumerate()
330 {
331 col_stats_set[index].null_count = file_column.null_count;
332 col_stats_set[index].max_value = file_column.max_value;
333 col_stats_set[index].min_value = file_column.min_value;
334 col_stats_set[index].sum_value = file_column.sum_value;
335 }
336
337 let conservative_num_rows = match num_rows {
342 Precision::Exact(nr) => nr,
343 _ => usize::MIN,
344 };
345 if conservative_num_rows <= limit.unwrap_or(usize::MAX) {
346 while let Some(current) = all_files.next().await {
347 let (mut file, file_stats) = current?;
348 file.statistics = Some(Arc::clone(&file_stats));
349 result_files.push(file);
350 if !collect_stats {
351 continue;
352 }
353
354 num_rows = num_rows.add(&file_stats.num_rows);
359
360 total_byte_size = total_byte_size.add(&file_stats.total_byte_size);
361
362 for (file_col_stats, col_stats) in file_stats
363 .column_statistics
364 .iter()
365 .zip(col_stats_set.iter_mut())
366 {
367 let ColumnStatistics {
368 null_count: file_nc,
369 max_value: file_max,
370 min_value: file_min,
371 sum_value: file_sum,
372 distinct_count: _,
373 } = file_col_stats;
374
375 col_stats.null_count = col_stats.null_count.add(file_nc);
376 col_stats.max_value = col_stats.max_value.max(file_max);
377 col_stats.min_value = col_stats.min_value.min(file_min);
378 col_stats.sum_value = col_stats.sum_value.add(file_sum);
379 }
380
381 if num_rows.get_value().unwrap_or(&usize::MIN)
386 > &limit.unwrap_or(usize::MAX)
387 {
388 break;
389 }
390 }
391 }
392 };
393
394 let mut statistics = Statistics {
395 num_rows,
396 total_byte_size,
397 column_statistics: col_stats_set,
398 };
399 if all_files.next().await.is_some() {
400 statistics = statistics.to_inexact()
404 }
405
406 Ok((result_files, statistics))
407}
408
409pub fn compute_file_group_statistics(
425 file_group: FileGroup,
426 file_schema: SchemaRef,
427 collect_stats: bool,
428) -> Result<FileGroup> {
429 if !collect_stats {
430 return Ok(file_group);
431 }
432
433 let file_group_stats = file_group.iter().filter_map(|file| {
434 let stats = file.statistics.as_ref()?;
435 Some(stats.as_ref())
436 });
437 let statistics = Statistics::try_merge_iter(file_group_stats, &file_schema)?;
438
439 Ok(file_group.with_statistics(Arc::new(statistics)))
440}
441
442pub fn compute_all_files_statistics(
460 file_groups: Vec<FileGroup>,
461 table_schema: SchemaRef,
462 collect_stats: bool,
463 inexact_stats: bool,
464) -> Result<(Vec<FileGroup>, Statistics)> {
465 let file_groups_with_stats = file_groups
466 .into_iter()
467 .map(|file_group| {
468 compute_file_group_statistics(
469 file_group,
470 Arc::clone(&table_schema),
471 collect_stats,
472 )
473 })
474 .collect::<Result<Vec<_>>>()?;
475
476 let file_groups_statistics = file_groups_with_stats
478 .iter()
479 .filter_map(|file_group| file_group.file_statistics(None));
480
481 let mut statistics =
482 Statistics::try_merge_iter(file_groups_statistics, &table_schema)?;
483
484 if inexact_stats {
485 statistics = statistics.to_inexact()
486 }
487
488 Ok((file_groups_with_stats, statistics))
489}
490
491#[deprecated(since = "47.0.0", note = "Use Statistics::add")]
492pub fn add_row_stats(
493 file_num_rows: Precision<usize>,
494 num_rows: Precision<usize>,
495) -> Precision<usize> {
496 file_num_rows.add(&num_rows)
497}