Skip to content
Open
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
6 changes: 3 additions & 3 deletions datafusion-examples/examples/relation_planner/table_sample.rs
Original file line number Diff line number Diff line change
Expand Up @@ -727,8 +727,8 @@ impl ExecutionPlan for SampleExec {
Some(self.metrics.clone_inner())
}

fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
let mut stats = self.input.partition_statistics(partition)?;
fn partition_statistics(&self, partition: Option<usize>) -> Result<Arc<Statistics>> {
let mut stats = Arc::unwrap_or_clone(self.input.partition_statistics(partition)?);
let ratio = self.upper_bound - self.lower_bound;

// Scale statistics by sampling ratio (inexact due to randomness)
Expand All @@ -741,7 +741,7 @@ impl ExecutionPlan for SampleExec {
.map(|n| (n as f64 * ratio) as usize)
.to_inexact();

Ok(stats)
Ok(Arc::new(stats))
}

fn apply_expressions(
Expand Down
8 changes: 4 additions & 4 deletions datafusion/core/tests/custom_sources_cases/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -184,12 +184,12 @@ impl ExecutionPlan for CustomExecutionPlan {
Ok(Box::pin(TestCustomRecordBatchStream { nb_batch: 1 }))
}

fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
fn partition_statistics(&self, partition: Option<usize>) -> Result<Arc<Statistics>> {
if partition.is_some() {
return Ok(Statistics::new_unknown(&self.schema()));
return Ok(Arc::new(Statistics::new_unknown(&self.schema())));
}
let batch = TEST_CUSTOM_RECORD_BATCH!().unwrap();
Ok(Statistics {
Ok(Arc::new(Statistics {
num_rows: Precision::Exact(batch.num_rows()),
total_byte_size: Precision::Absent,
column_statistics: self
Expand All @@ -208,7 +208,7 @@ impl ExecutionPlan for CustomExecutionPlan {
..Default::default()
})
.collect(),
})
}))
}

fn apply_expressions(
Expand Down
14 changes: 7 additions & 7 deletions datafusion/core/tests/custom_sources_cases/statistics.rs
Original file line number Diff line number Diff line change
Expand Up @@ -182,11 +182,11 @@ impl ExecutionPlan for StatisticsValidation {
unimplemented!("This plan only serves for testing statistics")
}

fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
fn partition_statistics(&self, partition: Option<usize>) -> Result<Arc<Statistics>> {
if partition.is_some() {
Ok(Statistics::new_unknown(&self.schema))
Ok(Arc::new(Statistics::new_unknown(&self.schema)))
} else {
Ok(self.stats.clone())
Ok(Arc::new(self.stats.clone()))
}
}

Expand Down Expand Up @@ -255,7 +255,7 @@ async fn sql_basic() -> Result<()> {
let physical_plan = df.create_physical_plan().await.unwrap();

// the statistics should be those of the source
assert_eq!(stats, physical_plan.partition_statistics(None)?);
assert_eq!(stats, *physical_plan.partition_statistics(None)?);

Ok(())
}
Expand Down Expand Up @@ -295,7 +295,7 @@ async fn sql_limit() -> Result<()> {
.collect(),
total_byte_size: Precision::Absent
},
physical_plan.partition_statistics(None)?
*physical_plan.partition_statistics(None)?
);

let df = ctx
Expand All @@ -304,7 +304,7 @@ async fn sql_limit() -> Result<()> {
.unwrap();
let physical_plan = df.create_physical_plan().await.unwrap();
// when the limit is larger than the original number of lines, statistics remain unchanged
assert_eq!(stats, physical_plan.partition_statistics(None)?);
assert_eq!(stats, *physical_plan.partition_statistics(None)?);

Ok(())
}
Expand All @@ -324,7 +324,7 @@ async fn sql_window() -> Result<()> {
let result = physical_plan.partition_statistics(None)?;

assert_eq!(stats.num_rows, result.num_rows);
let col_stats = result.column_statistics;
let col_stats = &result.column_statistics;
assert_eq!(2, col_stats.len());
assert_eq!(stats.column_statistics[1], col_stats[0]);

Expand Down
6 changes: 3 additions & 3 deletions datafusion/core/tests/physical_optimizer/join_selection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1191,12 +1191,12 @@ impl ExecutionPlan for StatisticsExec {
unimplemented!("This plan only serves for testing statistics")
}

fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
Ok(if partition.is_some() {
fn partition_statistics(&self, partition: Option<usize>) -> Result<Arc<Statistics>> {
Ok(Arc::new(if partition.is_some() {
Statistics::new_unknown(&self.schema)
} else {
self.stats.clone()
})
}))
}

fn apply_expressions(
Expand Down
Loading