Skip to content

Commit f48dc72

Browse files
authored
Remove the statistics() api in execution plan (#20319)
The API has been marked as deprecated for about 10 months, and we're planning to do more [changes](#20184) for statistics. I think it's time to let it go.
1 parent 51e5c98 commit f48dc72

37 files changed

Lines changed: 7 additions & 202 deletions

datafusion-examples/examples/execution_monitoring/memory_pool_execution_plan.rs

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@ use datafusion::execution::{SendableRecordBatchStream, TaskContext};
3838
use datafusion::logical_expr::LogicalPlanBuilder;
3939
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
4040
use datafusion::physical_plan::{
41-
DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, Statistics,
41+
DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties,
4242
};
4343
use datafusion::prelude::*;
4444
use futures::stream::{StreamExt, TryStreamExt};
@@ -296,8 +296,4 @@ impl ExecutionPlan for BufferingExecutionPlan {
296296
}),
297297
)))
298298
}
299-
300-
fn statistics(&self) -> Result<Statistics> {
301-
Ok(Statistics::new_unknown(&self.schema))
302-
}
303299
}

datafusion/core/tests/custom_sources_cases/mod.rs

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -180,10 +180,6 @@ impl ExecutionPlan for CustomExecutionPlan {
180180
Ok(Box::pin(TestCustomRecordBatchStream { nb_batch: 1 }))
181181
}
182182

183-
fn statistics(&self) -> Result<Statistics> {
184-
self.partition_statistics(None)
185-
}
186-
187183
fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
188184
if partition.is_some() {
189185
return Ok(Statistics::new_unknown(&self.schema()));

datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ use datafusion::logical_expr::TableProviderFilterPushDown;
2929
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
3030
use datafusion::physical_plan::{
3131
DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties,
32-
SendableRecordBatchStream, Statistics,
32+
SendableRecordBatchStream,
3333
};
3434
use datafusion::prelude::*;
3535
use datafusion::scalar::ScalarValue;
@@ -149,12 +149,6 @@ impl ExecutionPlan for CustomPlan {
149149
})),
150150
)))
151151
}
152-
153-
fn statistics(&self) -> Result<Statistics> {
154-
// here we could provide more accurate statistics
155-
// but we want to test the filter pushdown not the CBOs
156-
Ok(Statistics::new_unknown(&self.schema()))
157-
}
158152
}
159153

160154
#[derive(Clone, Debug)]

datafusion/core/tests/custom_sources_cases/statistics.rs

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -181,10 +181,6 @@ impl ExecutionPlan for StatisticsValidation {
181181
unimplemented!("This plan only serves for testing statistics")
182182
}
183183

184-
fn statistics(&self) -> Result<Statistics> {
185-
Ok(self.stats.clone())
186-
}
187-
188184
fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
189185
if partition.is_some() {
190186
Ok(Statistics::new_unknown(&self.schema))

datafusion/core/tests/physical_optimizer/enforce_distribution.rs

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -67,8 +67,7 @@ use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr};
6767
use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec;
6868
use datafusion_physical_plan::union::UnionExec;
6969
use datafusion_physical_plan::{
70-
DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, Statistics,
71-
displayable,
70+
DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, displayable,
7271
};
7372
use insta::Settings;
7473

@@ -210,10 +209,6 @@ impl ExecutionPlan for SortRequiredExec {
210209
) -> Result<datafusion_physical_plan::SendableRecordBatchStream> {
211210
unreachable!();
212211
}
213-
214-
fn statistics(&self) -> Result<Statistics> {
215-
self.input.partition_statistics(None)
216-
}
217212
}
218213

219214
fn parquet_exec() -> Arc<DataSourceExec> {

datafusion/core/tests/physical_optimizer/join_selection.rs

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1176,10 +1176,6 @@ impl ExecutionPlan for StatisticsExec {
11761176
unimplemented!("This plan only serves for testing statistics")
11771177
}
11781178

1179-
fn statistics(&self) -> Result<Statistics> {
1180-
Ok(self.stats.clone())
1181-
}
1182-
11831179
fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
11841180
Ok(if partition.is_some() {
11851181
Statistics::new_unknown(&self.schema)

datafusion/core/tests/user_defined/user_defined_plan.rs

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -84,7 +84,7 @@ use datafusion::{
8484
physical_expr::EquivalenceProperties,
8585
physical_plan::{
8686
DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
87-
PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics,
87+
PlanProperties, RecordBatchStream, SendableRecordBatchStream,
8888
},
8989
physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner},
9090
prelude::{SessionConfig, SessionContext},
@@ -742,12 +742,6 @@ impl ExecutionPlan for TopKExec {
742742
state: BTreeMap::new(),
743743
}))
744744
}
745-
746-
fn statistics(&self) -> Result<Statistics> {
747-
// to improve the optimizability of this plan
748-
// better statistics inference could be provided
749-
Ok(Statistics::new_unknown(&self.schema()))
750-
}
751745
}
752746

753747
// A very specialized TopK implementation

datafusion/datasource/src/source.rs

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -158,16 +158,6 @@ pub trait DataSource: Send + Sync + Debug {
158158
/// across all partitions if `partition` is `None`.
159159
fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics>;
160160

161-
/// Returns aggregate statistics across all partitions.
162-
///
163-
/// # Deprecated
164-
/// Use [`Self::partition_statistics`] instead, which provides more fine-grained
165-
/// control over statistics retrieval (per-partition or aggregate).
166-
#[deprecated(since = "51.0.0", note = "Use partition_statistics instead")]
167-
fn statistics(&self) -> Result<Statistics> {
168-
self.partition_statistics(None)
169-
}
170-
171161
/// Return a copy of this DataSource with a new fetch limit
172162
fn with_fetch(&self, _limit: Option<usize>) -> Option<Arc<dyn DataSource>>;
173163
fn fetch(&self) -> Option<usize>;

datafusion/ffi/src/execution_plan.rs

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -367,10 +367,6 @@ pub(crate) mod tests {
367367
) -> Result<SendableRecordBatchStream> {
368368
unimplemented!()
369369
}
370-
371-
fn statistics(&self) -> Result<datafusion::common::Statistics> {
372-
unimplemented!()
373-
}
374370
}
375371

376372
#[test]

datafusion/physical-optimizer/src/output_requirements.rs

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -244,10 +244,6 @@ impl ExecutionPlan for OutputRequirementExec {
244244
unreachable!();
245245
}
246246

247-
fn statistics(&self) -> Result<Statistics> {
248-
self.input.partition_statistics(None)
249-
}
250-
251247
fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
252248
self.input.partition_statistics(partition)
253249
}

0 commit comments

Comments
 (0)