-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Remove FilterExec from CoalesceBatches optimization rule, add fetch support #18630
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 11 commits
2921a29
038190b
44e760d
e27cfa1
4efb4f5
e557bf1
55ba3a3
65c3e94
389e8d7
2ba7570
af58682
4f6fec2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,7 +26,7 @@ use datafusion_common::error::Result; | |
| use datafusion_common::{config::ConfigOptions, internal_err}; | ||
| use datafusion_physical_expr::Partitioning; | ||
| use datafusion_physical_plan::{ | ||
| async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, | ||
| async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, | ||
| joins::HashJoinExec, repartition::RepartitionExec, ExecutionPlan, | ||
| }; | ||
|
|
||
|
|
@@ -60,8 +60,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { | |
| // wrap those ones with a CoalesceBatchesExec operator. An alternate approach here | ||
| // would be to build the coalescing logic directly into the operators | ||
| // See https://github.com/apache/datafusion/issues/139 | ||
| let wrap_in_coalesce = plan_any.downcast_ref::<FilterExec>().is_some() | ||
| || plan_any.downcast_ref::<HashJoinExec>().is_some() | ||
| let wrap_in_coalesce = plan_any.downcast_ref::<HashJoinExec>().is_some() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It looks like we only have
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes - I tried to apply this optimization to HashJoinExec would be a great candidate as it might be possible to avoid the |
||
| // Don't need to add CoalesceBatchesExec after a round robin RepartitionExec | ||
| || plan_any | ||
| .downcast_ref::<RepartitionExec>() | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,8 @@ use super::{ | |
| ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, | ||
| RecordBatchStream, SendableRecordBatchStream, Statistics, | ||
| }; | ||
| use crate::coalesce::LimitedBatchCoalescer; | ||
| use crate::coalesce::PushBatchStatus::LimitReached; | ||
| use crate::common::can_project; | ||
| use crate::execution_plan::CardinalityEffect; | ||
| use crate::filter_pushdown::{ | ||
|
|
@@ -42,7 +44,7 @@ use crate::{ | |
| DisplayFormatType, ExecutionPlan, | ||
| }; | ||
|
|
||
| use arrow::compute::{filter_record_batch, BatchCoalescer}; | ||
| use arrow::compute::filter_record_batch; | ||
| use arrow::datatypes::{DataType, SchemaRef}; | ||
| use arrow::record_batch::RecordBatch; | ||
| use datafusion_common::cast::as_boolean_array; | ||
|
|
@@ -87,6 +89,8 @@ pub struct FilterExec { | |
| projection: Option<Vec<usize>>, | ||
| /// Target batch size for output batches | ||
| batch_size: usize, | ||
| /// Number of rows to fetch | ||
| fetch: Option<usize>, | ||
| } | ||
|
|
||
| impl FilterExec { | ||
|
|
@@ -112,6 +116,7 @@ impl FilterExec { | |
| cache, | ||
| projection: None, | ||
| batch_size: FILTER_EXEC_DEFAULT_BATCH_SIZE, | ||
| fetch: None, | ||
| }) | ||
| } | ||
| other => { | ||
|
|
@@ -160,6 +165,7 @@ impl FilterExec { | |
| cache, | ||
| projection, | ||
| batch_size: self.batch_size, | ||
| fetch: self.fetch, | ||
| }) | ||
| } | ||
|
|
||
|
|
@@ -172,6 +178,7 @@ impl FilterExec { | |
| cache: self.cache.clone(), | ||
| projection: self.projection.clone(), | ||
| batch_size, | ||
| fetch: self.fetch, | ||
| }) | ||
| } | ||
|
|
||
|
|
@@ -351,7 +358,14 @@ impl DisplayAs for FilterExec { | |
| } else { | ||
| "".to_string() | ||
| }; | ||
| write!(f, "FilterExec: {}{}", self.predicate, display_projections) | ||
| let fetch = self | ||
| .fetch | ||
| .map_or_else(|| "".to_string(), |f| format!(", fetch={f}")); | ||
| write!( | ||
| f, | ||
| "FilterExec: {}{}{}", | ||
| self.predicate, display_projections, fetch | ||
| ) | ||
| } | ||
| DisplayFormatType::TreeRender => { | ||
| write!(f, "predicate={}", fmt_sql(self.predicate.as_ref())) | ||
|
|
@@ -393,7 +407,7 @@ impl ExecutionPlan for FilterExec { | |
| e.with_default_selectivity(selectivity) | ||
| }) | ||
| .and_then(|e| e.with_projection(self.projection().cloned())) | ||
| .map(|e| Arc::new(e) as _) | ||
| .map(|e| e.with_fetch(self.fetch).unwrap()) | ||
| } | ||
|
|
||
| fn execute( | ||
|
|
@@ -409,8 +423,11 @@ impl ExecutionPlan for FilterExec { | |
| input: self.input.execute(partition, context)?, | ||
| metrics, | ||
| projection: self.projection.clone(), | ||
| batch_coalescer: BatchCoalescer::new(self.schema(), self.batch_size) | ||
| .with_biggest_coalesce_batch_size(Some(self.batch_size / 2)), | ||
| batch_coalescer: LimitedBatchCoalescer::new( | ||
| self.schema(), | ||
Dandandan marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| self.batch_size, | ||
| self.fetch, | ||
| ), | ||
| })) | ||
| } | ||
|
|
||
|
|
@@ -569,6 +586,7 @@ impl ExecutionPlan for FilterExec { | |
| )?, | ||
| projection: None, | ||
| batch_size: self.batch_size, | ||
| fetch: self.fetch, | ||
| }; | ||
| Some(Arc::new(new) as _) | ||
| }; | ||
|
|
@@ -578,6 +596,19 @@ impl ExecutionPlan for FilterExec { | |
| updated_node, | ||
| }) | ||
| } | ||
|
|
||
| fn with_fetch(&self, fetch: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> { | ||
| Some(Arc::new(Self { | ||
| predicate: Arc::clone(&self.predicate), | ||
| input: Arc::clone(&self.input), | ||
| metrics: self.metrics.clone(), | ||
| default_selectivity: self.default_selectivity, | ||
| cache: self.cache.clone(), | ||
| projection: self.projection.clone(), | ||
| batch_size: self.batch_size, | ||
| fetch, | ||
| })) | ||
| } | ||
| } | ||
|
|
||
| impl EmbeddedProjection for FilterExec { | ||
|
|
@@ -648,7 +679,7 @@ struct FilterExecStream { | |
| /// The projection indices of the columns in the input schema | ||
| projection: Option<Vec<usize>>, | ||
| /// Batch coalescer to combine small batches | ||
| batch_coalescer: BatchCoalescer, | ||
| batch_coalescer: LimitedBatchCoalescer, | ||
| } | ||
|
|
||
| /// The metrics for `FilterExec` | ||
|
|
@@ -670,6 +701,23 @@ impl FilterExecMetrics { | |
| } | ||
| } | ||
|
|
||
| impl FilterExecStream { | ||
| fn flush_remaining_batches( | ||
| &mut self, | ||
| ) -> Poll<Option<std::result::Result<RecordBatch, DataFusionError>>> { | ||
| // Flush any remaining buffered batch | ||
| match self.batch_coalescer.finish() { | ||
| Ok(()) => { | ||
| Poll::Ready(self.batch_coalescer.next_completed_batch().map(|batch| { | ||
| self.metrics.selectivity.add_part(batch.num_rows()); | ||
| Ok(batch) | ||
| })) | ||
| } | ||
| Err(e) => Poll::Ready(Some(Err(e))), | ||
| } | ||
| } | ||
| } | ||
|
|
||
| pub fn batch_filter( | ||
| batch: &RecordBatch, | ||
| predicate: &Arc<dyn PhysicalExpr>, | ||
|
|
@@ -715,7 +763,7 @@ impl Stream for FilterExecStream { | |
| match ready!(self.input.poll_next_unpin(cx)) { | ||
| Some(Ok(batch)) => { | ||
| let timer = elapsed_compute.timer(); | ||
| self.predicate.as_ref() | ||
| let status = self.predicate.as_ref() | ||
| .evaluate(&batch) | ||
| .and_then(|v| v.into_array(batch.num_rows())) | ||
| .and_then(|array| { | ||
|
|
@@ -729,11 +777,11 @@ impl Stream for FilterExecStream { | |
| }).and_then(|(array, batch)| { | ||
| match as_boolean_array(&array) { | ||
| Ok(filter_array) => { | ||
| self.metrics.selectivity.add_part(filter_array.true_count()); | ||
| self.metrics.selectivity.add_total(batch.num_rows()); | ||
|
|
||
| self.batch_coalescer.push_batch_with_filter(batch.clone(), filter_array)?; | ||
| Ok(()) | ||
| // TODO: support push_batch_with_filter in LimitedBatchCoalescer | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is a good todo -- it would be good to file as a follow on ticket / PR perhaps I can do it if you like
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks, that would be nice |
||
| let batch = filter_record_batch(&batch, filter_array)?; | ||
| let state = self.batch_coalescer.push_batch(batch)?; | ||
| Ok(state) | ||
| } | ||
| Err(_) => { | ||
| internal_err!( | ||
|
|
@@ -742,28 +790,28 @@ impl Stream for FilterExecStream { | |
| } | ||
| } | ||
| })?; | ||
|
|
||
| timer.done(); | ||
|
|
||
| if self.batch_coalescer.has_completed_batch() { | ||
| poll = Poll::Ready(Some(Ok(self | ||
| .batch_coalescer | ||
| .next_completed_batch() | ||
| .expect("has_completed_batch is true")))); | ||
| if let LimitReached = status { | ||
| poll = self.flush_remaining_batches(); | ||
| break; | ||
| } | ||
|
|
||
| if let Some(batch) = self.batch_coalescer.next_completed_batch() { | ||
| self.metrics.selectivity.add_part(batch.num_rows()); | ||
| poll = Poll::Ready(Some(Ok(batch))); | ||
| break; | ||
| } | ||
| continue; | ||
| } | ||
| None => { | ||
| // Flush any remaining buffered batch | ||
| match self.batch_coalescer.finish_buffered_batch() { | ||
| match self.batch_coalescer.finish() { | ||
| Ok(()) => { | ||
| poll = Poll::Ready( | ||
| self.batch_coalescer.next_completed_batch().map(Ok), | ||
| ); | ||
| poll = self.flush_remaining_batches(); | ||
| } | ||
| Err(e) => { | ||
| poll = Poll::Ready(Some(Err(e.into()))); | ||
| poll = Poll::Ready(Some(Err(e))); | ||
| } | ||
| } | ||
| break; | ||
|
|
@@ -782,7 +830,6 @@ impl Stream for FilterExecStream { | |
| self.input.size_hint() | ||
| } | ||
| } | ||
|
|
||
| impl RecordBatchStream for FilterExecStream { | ||
| fn schema(&self) -> SchemaRef { | ||
| Arc::clone(&self.schema) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This comment could probably be removed now (the ticket is long since closed)