-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Add option to FilterExec to prevent re-using input batches #12039
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 all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -15,11 +15,6 @@ | |
| // specific language governing permissions and limitations | ||
| // under the License. | ||
|
|
||
| use std::any::Any; | ||
| use std::pin::Pin; | ||
| use std::sync::Arc; | ||
| use std::task::{ready, Context, Poll}; | ||
|
|
||
| use super::{ | ||
| ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, | ||
| RecordBatchStream, SendableRecordBatchStream, Statistics, | ||
|
|
@@ -28,10 +23,16 @@ use crate::{ | |
| metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, | ||
| DisplayFormatType, ExecutionPlan, | ||
| }; | ||
| use arrow::array::MutableArrayData; | ||
| use std::any::Any; | ||
| use std::pin::Pin; | ||
| use std::sync::Arc; | ||
| use std::task::{ready, Context, Poll}; | ||
|
|
||
| use arrow::compute::filter_record_batch; | ||
| use arrow::datatypes::{DataType, SchemaRef}; | ||
| use arrow::record_batch::RecordBatch; | ||
| use arrow_array::{make_array, ArrayRef, RecordBatchOptions}; | ||
| use datafusion_common::cast::as_boolean_array; | ||
| use datafusion_common::stats::Precision; | ||
| use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; | ||
|
|
@@ -61,13 +62,25 @@ pub struct FilterExec { | |
| default_selectivity: u8, | ||
| /// Properties equivalence properties, partitioning, etc. | ||
| cache: PlanProperties, | ||
| /// Whether to allow an input batch to be returned unmodified in the case where | ||
| /// the predicate evaluates to true for all rows in the batch | ||
| reuse_input_batches: bool, | ||
| } | ||
|
|
||
| impl FilterExec { | ||
| /// Create a FilterExec on an input | ||
| pub fn try_new( | ||
| predicate: Arc<dyn PhysicalExpr>, | ||
| input: Arc<dyn ExecutionPlan>, | ||
| ) -> Result<Self> { | ||
| Self::try_new_with_reuse_input_batches(predicate, input, true) | ||
| } | ||
|
|
||
| /// Create a FilterExec on an input using the specified kernel to create filtered batches | ||
| pub fn try_new_with_reuse_input_batches( | ||
| predicate: Arc<dyn PhysicalExpr>, | ||
| input: Arc<dyn ExecutionPlan>, | ||
| reuse_input_batches: bool, | ||
| ) -> Result<Self> { | ||
| match predicate.data_type(input.schema().as_ref())? { | ||
| DataType::Boolean => { | ||
|
|
@@ -80,6 +93,7 @@ impl FilterExec { | |
| metrics: ExecutionPlanMetricsSet::new(), | ||
| default_selectivity, | ||
| cache, | ||
| reuse_input_batches, | ||
| }) | ||
| } | ||
| other => { | ||
|
|
@@ -283,6 +297,7 @@ impl ExecutionPlan for FilterExec { | |
| predicate: Arc::clone(&self.predicate), | ||
| input: self.input.execute(partition, context)?, | ||
| baseline_metrics, | ||
| reuse_input_batches: self.reuse_input_batches, | ||
| })) | ||
| } | ||
|
|
||
|
|
@@ -345,19 +360,55 @@ struct FilterExecStream { | |
| input: SendableRecordBatchStream, | ||
| /// runtime metrics recording | ||
| baseline_metrics: BaselineMetrics, | ||
| /// Whether to allow an input batch to be returned unmodified in the case where | ||
| /// the predicate evaluates to true for all rows in the batch | ||
| reuse_input_batches: bool, | ||
| } | ||
|
|
||
| pub(crate) fn batch_filter( | ||
| batch: &RecordBatch, | ||
| predicate: &Arc<dyn PhysicalExpr>, | ||
| reuse_input_batches: bool, | ||
| ) -> Result<RecordBatch> { | ||
| predicate | ||
| .evaluate(batch) | ||
| .and_then(|v| v.into_array(batch.num_rows())) | ||
| .and_then(|array| { | ||
| Ok(match as_boolean_array(&array) { | ||
| // apply filter array to record batch | ||
| Ok(filter_array) => filter_record_batch(batch, filter_array)?, | ||
| Ok(filter_array) => { | ||
| if reuse_input_batches { | ||
| filter_record_batch(batch, filter_array)? | ||
| } else { | ||
| if filter_array.true_count() == batch.num_rows() { | ||
| // special case where we just make an exact copy | ||
|
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. I think I am missing something -- why go through all the effort with In other words, why isn't thus simply Ok(batch.clone())To literally return the input batch? I think that would be less code and faster
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. That would be the same that the filter. The use case as far as I understand is that for comet the data needs to be a new copy, as spark will reuse the existing data/arrays.
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.
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. I see -- I think it would help if we made a function with a name that made it clear what was going on, something like fn force_new_data_copy(...)I also left a suggestion here apache/datafusion-comet#835 (comment) |
||
| let arrays: Vec<ArrayRef> = batch | ||
| .columns() | ||
| .iter() | ||
| .map(|array| { | ||
| let capacity = array.len(); | ||
| let data = array.to_data(); | ||
| let mut mutable = MutableArrayData::new( | ||
| vec![&data], | ||
| false, | ||
| capacity, | ||
| ); | ||
| mutable.extend(0, 0, capacity); | ||
| make_array(mutable.freeze()) | ||
| }) | ||
| .collect(); | ||
| let options = RecordBatchOptions::new() | ||
| .with_row_count(Some(batch.num_rows())); | ||
| RecordBatch::try_new_with_options( | ||
| batch.schema().clone(), | ||
| arrays, | ||
| &options, | ||
| )? | ||
| } else { | ||
| filter_record_batch(batch, filter_array)? | ||
| } | ||
| } | ||
| } | ||
| Err(_) => { | ||
| return internal_err!( | ||
| "Cannot create filter_array from non-boolean predicates" | ||
|
|
@@ -379,7 +430,8 @@ impl Stream for FilterExecStream { | |
| match ready!(self.input.poll_next_unpin(cx)) { | ||
| Some(Ok(batch)) => { | ||
| let timer = self.baseline_metrics.elapsed_compute().timer(); | ||
| let filtered_batch = batch_filter(&batch, &self.predicate)?; | ||
| let filtered_batch = | ||
|
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. I recommend adding a test so we don't accidentally break the feature by accident |
||
| batch_filter(&batch, &self.predicate, self.reuse_input_batches)?; | ||
| timer.done(); | ||
| // skip entirely filtered batches | ||
| if filtered_batch.num_rows() == 0 { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
As computing true count is not free, I am wondering if we can either
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.
Specifically, I think the
optimizefunction would be a natural place to put this: https://docs.rs/arrow-select/52.2.0/src/arrow_select/filter.rs.html#181