Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
215 changes: 214 additions & 1 deletion datafusion/catalog-listing/src/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,8 @@ use async_trait::async_trait;
use datafusion_catalog::{ScanArgs, ScanResult, Session, TableProvider};
use datafusion_common::stats::Precision;
use datafusion_common::{
Constraints, SchemaExt, Statistics, internal_datafusion_err, plan_err, project_schema,
Constraints, ScalarValue, SchemaExt, Statistics, internal_datafusion_err, plan_err,
project_schema,
};
use datafusion_datasource::file::FileSource;
use datafusion_datasource::file_groups::FileGroup;
Expand All @@ -39,6 +40,9 @@ use datafusion_execution::cache::cache_manager::FileStatisticsCache;
use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache;
use datafusion_expr::dml::InsertOp;
use datafusion_expr::execution_props::ExecutionProps;
use datafusion_expr::statistics::{
SatisfiedStatistics, StatisticsRequest, StatisticsValue,
};
use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType};
use datafusion_physical_expr::create_lex_ordering;
use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory;
Expand Down Expand Up @@ -515,6 +519,39 @@ impl TableProvider for ListingTable {
.list_files_for_scan(state, &partition_filters, statistic_file_limit)
.await?;

// If the caller asked for per-file stats and we already have dense
// per-file `Statistics` (i.e. `collect_statistics=true` populated
// them during file listing), project them into a sparse
// `satisfied_stats` map keyed by request. This is "free" — the IO
// / decode cost was already paid; we're just surfacing what we
// have in the request-driven shape that downstream consumers
// (e.g. `FilePruner` via `SparseFilePruningStats`) prefer.
if !args.statistics_requests().is_empty() && self.options.collect_stat {
partitioned_file_lists = partitioned_file_lists
.into_iter()
.map(|fg| {
let files = fg
.into_inner()
.into_iter()
.map(|pf| {
let sparse = pf.statistics.as_ref().map(|stats| {
project_satisfied_stats(
args.statistics_requests(),
&self.table_schema,
stats,
)
});
match sparse {
Some(s) => pf.with_satisfied_stats(Arc::new(s)),
None => pf,
}
})
.collect();
FileGroup::new(files)
})
.collect();
}

// if no files need to be read, return an `EmptyExec`
if partitioned_file_lists.is_empty() {
let projected_schema = project_schema(&self.schema(), projection.as_ref())?;
Expand Down Expand Up @@ -688,6 +725,108 @@ impl TableProvider for ListingTable {
}
}

/// Project a per-file dense [`Statistics`] into the sparse map shape
/// that downstream consumers (e.g. `FilePruner` via
/// `SparseFilePruningStats`) read. Only entries the request list asked
/// for are present; non-`Absent` slots are kept verbatim (preserves the
/// `Precision` already on the field). The dense `Statistics` itself is
/// untouched.
///
/// `DistinctCount` / `Sum` / `ByteSize` will only be present if the
/// underlying file format actually populated them on `ColumnStatistics`
/// — for parquet, only `null_count`, `min_value`, `max_value`, and
/// `num_rows` are typically available from thrift footers; the rest
/// stay `Absent` (so the entry is omitted).
fn project_satisfied_stats(
requests: &[StatisticsRequest],
schema: &SchemaRef,
stats: &Statistics,
) -> SatisfiedStatistics {
let mut out = SatisfiedStatistics::new();
for req in requests {
let value = match req {
StatisticsRequest::RowCount => precision_to_scalar(&stats.num_rows, |n| {
ScalarValue::UInt64(Some(n as u64))
}),
StatisticsRequest::TotalByteSize => {
precision_to_scalar(&stats.total_byte_size, |n| {
ScalarValue::UInt64(Some(n as u64))
})
}
StatisticsRequest::Min(c) => {
column_scalar(schema, stats, &c.name, |cs| &cs.min_value)
}
StatisticsRequest::Max(c) => {
column_scalar(schema, stats, &c.name, |cs| &cs.max_value)
}
StatisticsRequest::Sum(c) => {
column_scalar(schema, stats, &c.name, |cs| &cs.sum_value)
}
StatisticsRequest::NullCount(c) => {
column_usize(schema, stats, &c.name, |cs| &cs.null_count)
}
StatisticsRequest::DistinctCount(c) => {
column_usize(schema, stats, &c.name, |cs| &cs.distinct_count)
}
StatisticsRequest::ByteSize(c) => {
column_usize(schema, stats, &c.name, |cs| &cs.byte_size)
}
};
// Skip `Absent` entries — the sparse map's invariant is "only
// entries the provider answered". Consumers infer `Absent` from
// a missing key.
if !matches!(value, StatisticsValue::Absent) {
out.insert(req.clone(), value);
}
}
out
}

fn precision_to_scalar(
p: &Precision<usize>,
map: impl Fn(usize) -> ScalarValue,
) -> StatisticsValue {
match p {
Precision::Exact(n) => StatisticsValue::Scalar(Precision::Exact(map(*n))),
Precision::Inexact(n) => StatisticsValue::Scalar(Precision::Inexact(map(*n))),
Precision::Absent => StatisticsValue::Absent,
}
}

fn column_scalar(
schema: &SchemaRef,
stats: &Statistics,
name: &str,
pick: impl Fn(&datafusion_common::ColumnStatistics) -> &Precision<ScalarValue>,
) -> StatisticsValue {
let Ok(idx) = schema.index_of(name) else {
return StatisticsValue::Absent;
};
let Some(cs) = stats.column_statistics.get(idx) else {
return StatisticsValue::Absent;
};
match pick(cs) {
Precision::Exact(v) => StatisticsValue::Scalar(Precision::Exact(v.clone())),
Precision::Inexact(v) => StatisticsValue::Scalar(Precision::Inexact(v.clone())),
Precision::Absent => StatisticsValue::Absent,
}
}

fn column_usize(
schema: &SchemaRef,
stats: &Statistics,
name: &str,
pick: impl Fn(&datafusion_common::ColumnStatistics) -> &Precision<usize>,
) -> StatisticsValue {
let Ok(idx) = schema.index_of(name) else {
return StatisticsValue::Absent;
};
let Some(cs) = stats.column_statistics.get(idx) else {
return StatisticsValue::Absent;
};
precision_to_scalar(pick(cs), |n| ScalarValue::UInt64(Some(n as u64)))
}

impl ListingTable {
/// Get the list of files for a scan as well as the file level statistics.
/// The list is grouped to let the execution plan know how the files should
Expand Down Expand Up @@ -1049,4 +1188,78 @@ mod tests {
let result = derive_common_ordering_from_files(&file_groups);
assert_eq!(result, Some(ordering));
}

#[test]
fn project_satisfied_stats_picks_only_requested_and_present() {
use arrow::datatypes::{DataType, Field, Schema};
use datafusion_common::{Column, ColumnStatistics};
use datafusion_expr::statistics::{StatisticsRequest, StatisticsValue};

let schema: SchemaRef = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int64, true),
Field::new("b", DataType::Int64, true),
]));

let mut stats = Statistics::new_unknown(&schema);
stats.num_rows = Precision::Exact(42);
// a: min/max known, null_count Absent
stats.column_statistics[0] = ColumnStatistics {
null_count: Precision::Absent,
max_value: Precision::Exact(ScalarValue::Int64(Some(10))),
min_value: Precision::Exact(ScalarValue::Int64(Some(1))),
sum_value: Precision::Absent,
distinct_count: Precision::Absent,
byte_size: Precision::Absent,
};
// b: only null_count known
stats.column_statistics[1] = ColumnStatistics {
null_count: Precision::Exact(3),
max_value: Precision::Absent,
min_value: Precision::Absent,
sum_value: Precision::Absent,
distinct_count: Precision::Absent,
byte_size: Precision::Absent,
};

let a = Column::from_name("a");
let b = Column::from_name("b");
let requests = vec![
StatisticsRequest::RowCount,
StatisticsRequest::Min(a.clone()),
StatisticsRequest::Max(a.clone()),
StatisticsRequest::NullCount(a.clone()), // Absent — should be omitted
StatisticsRequest::NullCount(b.clone()),
StatisticsRequest::Min(b.clone()), // Absent — should be omitted
];

let out = project_satisfied_stats(&requests, &schema, &stats);

assert_eq!(out.len(), 4);
assert!(matches!(
out.get(&StatisticsRequest::RowCount),
Some(StatisticsValue::Scalar(Precision::Exact(
ScalarValue::UInt64(Some(42))
)))
));
assert!(matches!(
out.get(&StatisticsRequest::Min(a.clone())),
Some(StatisticsValue::Scalar(Precision::Exact(
ScalarValue::Int64(Some(1))
)))
));
assert!(matches!(
out.get(&StatisticsRequest::Max(a.clone())),
Some(StatisticsValue::Scalar(Precision::Exact(
ScalarValue::Int64(Some(10))
)))
));
assert!(matches!(
out.get(&StatisticsRequest::NullCount(b.clone())),
Some(StatisticsValue::Scalar(Precision::Exact(
ScalarValue::UInt64(Some(3))
)))
));
assert!(!out.contains_key(&StatisticsRequest::NullCount(a)));
assert!(!out.contains_key(&StatisticsRequest::Min(b)));
}
}
28 changes: 28 additions & 0 deletions datafusion/catalog/src/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ use async_trait::async_trait;
use datafusion_common::{Constraints, Statistics, not_impl_err};
use datafusion_common::{Result, internal_err};
use datafusion_expr::Expr;
use datafusion_expr::statistics::StatisticsRequest;

use datafusion_expr::dml::InsertOp;
use datafusion_expr::{
Expand Down Expand Up @@ -406,6 +407,7 @@ pub struct ScanArgs<'a> {
filters: Option<&'a [Expr]>,
projection: Option<&'a [usize]>,
limit: Option<usize>,
statistics_requests: &'a [StatisticsRequest],
}

impl<'a> ScanArgs<'a> {
Expand Down Expand Up @@ -467,6 +469,32 @@ impl<'a> ScanArgs<'a> {
pub fn limit(&self) -> Option<usize> {
self.limit
}

/// Set a list of statistics the caller would like the provider to
/// answer if it can do so cheaply.
///
/// Typical sources a provider may answer from:
/// * Parquet thrift footers (Min/Max/NullCount/RowCount, exact)
/// * An external metadata catalog (Delta/Iceberg/Hudi manifests,
/// Hive-Metastore-style stats columns)
/// * Cached / materialized stats columns
///
/// Providers surface their answers on a per-file basis via
/// `PartitionedFile::satisfied_stats` (a sparse map keyed by the
/// same `StatisticsRequest`). Anything the provider can't answer
/// is simply omitted — consumers treat a missing key as "absent".
/// The contract is "answer what's free, leave the rest off the
/// map": providers MUST NOT do expensive scans purely to satisfy
/// these requests.
pub fn with_statistics_requests(mut self, requests: &'a [StatisticsRequest]) -> Self {
self.statistics_requests = requests;
self
}

/// Get the statistics requests. Empty if none were set.
pub fn statistics_requests(&self) -> &'a [StatisticsRequest] {
self.statistics_requests
}
}

/// Result of a table scan operation from [`TableProvider::scan_with_args`].
Expand Down
Loading
Loading