Skip to content

Commit 3aebce8

Browse files
committed
Apply bounds to the provided input statistics when a filter is present in the plan
1 parent 36c8789 commit 3aebce8

File tree

4 files changed

+145
-24
lines changed

4 files changed

+145
-24
lines changed

crates/integration_tests/tests/datafusion.rs

Lines changed: 41 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ use datafusion::assert_batches_eq;
2424
use datafusion::catalog::TableProvider;
2525
use datafusion::common::stats::Precision;
2626
use datafusion::common::{ColumnStatistics, ScalarValue, Statistics};
27+
use datafusion::logical_expr::{col, lit};
2728
use datafusion::prelude::SessionContext;
2829
use iceberg::{Catalog, Result, TableIdent};
2930
use iceberg_datafusion::IcebergTableProvider;
@@ -38,16 +39,11 @@ async fn test_basic_queries() -> Result<()> {
3839

3940
let table = catalog
4041
.load_table(&TableIdent::from_strs(["default", "types_test"]).unwrap())
41-
.await
42-
.unwrap();
42+
.await?;
4343

4444
let ctx = SessionContext::new();
4545

46-
let table_provider = Arc::new(
47-
IcebergTableProvider::try_new_from_table(table)
48-
.await
49-
.unwrap(),
50-
);
46+
let table_provider = Arc::new(IcebergTableProvider::try_new_from_table(table).await?);
5147

5248
let schema = table_provider.schema();
5349

@@ -146,22 +142,23 @@ async fn test_statistics() -> Result<()> {
146142

147143
let catalog = fixture.rest_catalog;
148144

145+
// Test table statistics
149146
let table = catalog
150-
.load_table(
151-
&TableIdent::from_strs(["default", "test_positional_merge_on_read_double_deletes"])
152-
.unwrap(),
153-
)
154-
.await
155-
.unwrap();
147+
.load_table(&TableIdent::from_strs([
148+
"default",
149+
"test_positional_merge_on_read_double_deletes",
150+
])?)
151+
.await?;
156152

157-
let stats = IcebergTableProvider::try_new_from_table(table)
153+
let table_provider = IcebergTableProvider::try_new_from_table(table)
158154
.await?
159155
.with_computed_statistics()
160-
.await
161-
.statistics();
156+
.await;
157+
158+
let table_stats = table_provider.statistics();
162159

163160
assert_eq!(
164-
stats,
161+
table_stats,
165162
Some(Statistics {
166163
num_rows: Precision::Inexact(12),
167164
total_byte_size: Precision::Absent,
@@ -188,5 +185,32 @@ async fn test_statistics() -> Result<()> {
188185
})
189186
);
190187

188+
// Test plan statistics with filtering
189+
let ctx = SessionContext::new();
190+
let scan = table_provider
191+
.scan(
192+
&ctx.state(),
193+
Some(&vec![1]),
194+
&[col("number").gt(lit(4))],
195+
None,
196+
)
197+
.await
198+
.unwrap();
199+
200+
let plan_stats = scan.statistics().unwrap();
201+
202+
// The estimate for the number of rows and the min value for the column are changed in response
203+
// to the filtration
204+
assert_eq!(plan_stats, Statistics {
205+
num_rows: Precision::Inexact(8),
206+
total_byte_size: Precision::Absent,
207+
column_statistics: vec![ColumnStatistics {
208+
null_count: Precision::Inexact(0),
209+
max_value: Precision::Inexact(ScalarValue::Int32(Some(12))),
210+
min_value: Precision::Inexact(ScalarValue::Int32(Some(5))),
211+
distinct_count: Precision::Absent,
212+
},],
213+
});
214+
191215
Ok(())
192216
}

crates/integrations/datafusion/src/physical_plan/scan.rs

Lines changed: 44 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,20 +22,22 @@ use std::vec;
2222

2323
use datafusion::arrow::array::RecordBatch;
2424
use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef;
25-
use datafusion::common::Statistics;
25+
use datafusion::common::{Statistics, ToDFSchema};
2626
use datafusion::error::Result as DFResult;
2727
use datafusion::execution::{SendableRecordBatchStream, TaskContext};
28-
use datafusion::physical_expr::EquivalenceProperties;
28+
use datafusion::logical_expr::utils::conjunction;
29+
use datafusion::physical_expr::{create_physical_expr, EquivalenceProperties};
2930
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
3031
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
3132
use datafusion::physical_plan::{DisplayAs, ExecutionPlan, Partitioning, PlanProperties};
3233
use datafusion::prelude::Expr;
3334
use futures::{Stream, TryStreamExt};
3435
use iceberg::expr::Predicate;
3536
use iceberg::table::Table;
37+
use log::warn;
3638

3739
use super::expr_to_predicate::convert_filters_to_predicate;
38-
use crate::to_datafusion_error;
40+
use crate::{apply_bounds, to_datafusion_error};
3941

4042
/// Manages the scanning process of an Iceberg [`Table`], encapsulating the
4143
/// necessary details and computed properties required for execution planning.
@@ -63,14 +65,35 @@ impl IcebergTableScan {
6365
table: Table,
6466
snapshot_id: Option<i64>,
6567
schema: ArrowSchemaRef,
66-
statistics: Statistics,
68+
statistics: Option<Statistics>,
6769
projection: Option<&Vec<usize>>,
6870
filters: &[Expr],
6971
) -> Self {
7072
let output_schema = match projection {
7173
None => schema.clone(),
7274
Some(projection) => Arc::new(schema.project(projection).unwrap()),
7375
};
76+
77+
let statistics = statistics
78+
.as_ref()
79+
.map(|stats| {
80+
let stats = match projection {
81+
None => stats.clone(),
82+
Some(projection) => stats.clone().project(Some(projection)),
83+
};
84+
Self::bound_statistics(stats.clone(), filters, output_schema.clone())
85+
})
86+
.transpose()
87+
.inspect_err(|err| {
88+
warn!(
89+
"Failed to bound input statistics, defaulting to none: {:?}",
90+
err
91+
)
92+
})
93+
.ok()
94+
.flatten()
95+
.unwrap_or(Statistics::new_unknown(output_schema.as_ref()));
96+
7497
let plan_properties = Self::compute_properties(output_schema.clone());
7598
let projection = get_column_names(schema.clone(), projection);
7699
let predicates = convert_filters_to_predicate(filters);
@@ -97,6 +120,23 @@ impl IcebergTableScan {
97120
Boundedness::Bounded,
98121
)
99122
}
123+
124+
/// Estimate the effective bounded statistics corresponding to the provided filter expressions
125+
fn bound_statistics(
126+
input_stats: Statistics,
127+
filters: &[Expr],
128+
schema: ArrowSchemaRef,
129+
) -> DFResult<Statistics> {
130+
Ok(if let Some(filters) = conjunction(filters.to_vec()) {
131+
let schema = schema.clone();
132+
let df_schema = schema.clone().to_dfschema()?;
133+
let predicate = create_physical_expr(&filters, &df_schema, &Default::default())?;
134+
135+
apply_bounds(input_stats, &predicate, schema)?
136+
} else {
137+
input_stats
138+
})
139+
}
100140
}
101141

102142
impl ExecutionPlan for IcebergTableScan {

crates/integrations/datafusion/src/statistics.rs

Lines changed: 59 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,9 +16,13 @@
1616
// under the License.
1717

1818
use std::collections::HashMap;
19+
use std::sync::Arc;
1920

21+
use datafusion::arrow::datatypes::SchemaRef;
2022
use datafusion::common::stats::Precision;
2123
use datafusion::common::{ColumnStatistics, Statistics};
24+
use datafusion::error::Result as DFResult;
25+
use datafusion::physical_expr::{analyze, AnalysisContext, ExprBoundaries, PhysicalExpr};
2226
use iceberg::spec::{DataContentType, ManifestStatus};
2327
use iceberg::table::Table;
2428
use iceberg::Result;
@@ -113,3 +117,58 @@ pub async fn compute_statistics(table: &Table, snapshot_id: Option<i64>) -> Resu
113117
column_statistics: col_stats,
114118
})
115119
}
120+
121+
// Apply bounds to the provided input statistics.
122+
//
123+
// Adapted from `FilterExec::statistics_helper` in DataFusion.
124+
pub fn apply_bounds(
125+
input_stats: Statistics,
126+
predicate: &Arc<dyn PhysicalExpr>,
127+
schema: SchemaRef,
128+
) -> DFResult<Statistics> {
129+
let num_rows = input_stats.num_rows;
130+
let total_byte_size = input_stats.total_byte_size;
131+
let input_analysis_ctx =
132+
AnalysisContext::try_from_statistics(&schema, &input_stats.column_statistics)?;
133+
134+
let analysis_ctx = analyze(predicate, input_analysis_ctx, &schema)?;
135+
136+
// Estimate (inexact) selectivity of predicate
137+
let selectivity = analysis_ctx.selectivity.unwrap_or(1.0);
138+
let num_rows = num_rows.with_estimated_selectivity(selectivity);
139+
let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity);
140+
141+
let column_statistics = analysis_ctx
142+
.boundaries
143+
.into_iter()
144+
.enumerate()
145+
.map(
146+
|(
147+
idx,
148+
ExprBoundaries {
149+
interval,
150+
distinct_count,
151+
..
152+
},
153+
)| {
154+
let (lower, upper) = interval.into_bounds();
155+
let (min_value, max_value) = if lower.eq(&upper) {
156+
(Precision::Exact(lower), Precision::Exact(upper))
157+
} else {
158+
(Precision::Inexact(lower), Precision::Inexact(upper))
159+
};
160+
ColumnStatistics {
161+
null_count: input_stats.column_statistics[idx].null_count.to_inexact(),
162+
max_value,
163+
min_value,
164+
distinct_count: distinct_count.to_inexact(),
165+
}
166+
},
167+
)
168+
.collect();
169+
Ok(Statistics {
170+
num_rows,
171+
total_byte_size,
172+
column_statistics,
173+
})
174+
}

crates/integrations/datafusion/src/table/mod.rs

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -152,9 +152,7 @@ impl TableProvider for IcebergTableProvider {
152152
self.table.clone(),
153153
self.snapshot_id,
154154
self.schema.clone(),
155-
self.statistics
156-
.clone()
157-
.unwrap_or(Statistics::new_unknown(self.schema.as_ref())),
155+
self.statistics.clone(),
158156
projection,
159157
filters,
160158
)))

0 commit comments

Comments
 (0)