-
Notifications
You must be signed in to change notification settings - Fork 2.1k
Redesign the try_reverse_output to support more cases #19446
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 15 commits
585abd0
d729a6c
d2bafa6
4218989
a2e9360
6be7de2
b92ffc3
8ab438b
6ed8dad
5c04b65
5a9bfaf
31f77d1
a8ca48c
90d0ce5
a5489d5
0ad8b86
31d64e0
e6ab4e0
96d6ea7
d9eda5d
70e5cfb
ba8f240
22014d1
237746e
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 |
|---|---|---|
|
|
@@ -82,7 +82,7 @@ use crate::filter_pushdown::{ | |
| use crate::projection::ProjectionExec; | ||
| use crate::{ | ||
| DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream, | ||
| SendableRecordBatchStream, | ||
| SendableRecordBatchStream, SortOrderPushdownResult, | ||
| }; | ||
| use arrow::record_batch::RecordBatch; | ||
| use arrow_schema::Schema; | ||
|
|
@@ -91,6 +91,7 @@ use datafusion_execution::TaskContext; | |
|
|
||
| use crate::execution_plan::SchedulingType; | ||
| use crate::stream::RecordBatchStreamAdapter; | ||
| use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; | ||
| use futures::{Stream, StreamExt}; | ||
|
|
||
| /// A stream that passes record batches through unchanged while cooperating with the Tokio runtime. | ||
|
|
@@ -328,6 +329,27 @@ impl ExecutionPlan for CooperativeExec { | |
| ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> { | ||
| Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) | ||
| } | ||
|
|
||
| fn try_pushdown_sort( | ||
|
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 seems like a good improvement but is orthogonal to the change to use |
||
| &self, | ||
| order: &[PhysicalSortExpr], | ||
| ) -> Result<SortOrderPushdownResult<Arc<dyn ExecutionPlan>>> { | ||
| let child = self.input(); | ||
|
|
||
| match child.try_pushdown_sort(order)? { | ||
| SortOrderPushdownResult::Exact { inner } => { | ||
| let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; | ||
| Ok(SortOrderPushdownResult::Exact { inner: new_exec }) | ||
| } | ||
| SortOrderPushdownResult::Inexact { inner } => { | ||
| let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; | ||
| Ok(SortOrderPushdownResult::Inexact { inner: new_exec }) | ||
| } | ||
| SortOrderPushdownResult::Unsupported => { | ||
| Ok(SortOrderPushdownResult::Unsupported) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /// Creates a [`CooperativeStream`] wrapper around the given [`RecordBatchStream`]. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,7 +24,7 @@ use super::expressions::{Column, Literal}; | |
| use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; | ||
| use super::{ | ||
| DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, | ||
| SendableRecordBatchStream, Statistics, | ||
| SendableRecordBatchStream, SortOrderPushdownResult, Statistics, | ||
| }; | ||
| use crate::execution_plan::CardinalityEffect; | ||
| use crate::filter_pushdown::{ | ||
|
|
@@ -51,7 +51,9 @@ use datafusion_physical_expr::equivalence::ProjectionMapping; | |
| use datafusion_physical_expr::projection::Projector; | ||
| use datafusion_physical_expr::utils::collect_columns; | ||
| use datafusion_physical_expr_common::physical_expr::{PhysicalExprRef, fmt_sql}; | ||
| use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; | ||
| use datafusion_physical_expr_common::sort_expr::{ | ||
| LexOrdering, LexRequirement, PhysicalSortExpr, | ||
| }; | ||
| // Re-exported from datafusion-physical-expr for backwards compatibility | ||
| // We recommend updating your imports to use datafusion-physical-expr directly | ||
| pub use datafusion_physical_expr::projection::{ | ||
|
|
@@ -357,6 +359,27 @@ impl ExecutionPlan for ProjectionExec { | |
| ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> { | ||
| Ok(FilterPushdownPropagation::if_all(child_pushdown_result)) | ||
| } | ||
|
|
||
| fn try_pushdown_sort( | ||
| &self, | ||
| order: &[PhysicalSortExpr], | ||
|
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. What if the sort depends on the output of the projection? Presumably then the projection should say the sort can't be pushed down?
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. Good point @adriangb ! I will try to address this and add more tests. |
||
| ) -> Result<SortOrderPushdownResult<Arc<dyn ExecutionPlan>>> { | ||
| let child = self.input(); | ||
|
|
||
| match child.try_pushdown_sort(order)? { | ||
| SortOrderPushdownResult::Exact { inner } => { | ||
| let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; | ||
| Ok(SortOrderPushdownResult::Exact { inner: new_exec }) | ||
| } | ||
| SortOrderPushdownResult::Inexact { inner } => { | ||
| let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?; | ||
| Ok(SortOrderPushdownResult::Inexact { inner: new_exec }) | ||
| } | ||
| SortOrderPushdownResult::Unsupported => { | ||
| Ok(SortOrderPushdownResult::Unsupported) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| impl ProjectionStream { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.