-
Notifications
You must be signed in to change notification settings - Fork 2k
feat(cubestore): push Sort(fetch=N) to workers for ORDER BY group-by … #11027
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
base: master
Are you sure you want to change the base?
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 |
|---|---|---|
|
|
@@ -5,16 +5,19 @@ | |
| pub mod rewrite_plan; | ||
| pub mod rolling_optimizer; | ||
| mod trace_data_loaded; | ||
|
|
||
|
Check warning on line 8 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| use super::serialized_plan::PreSerializedPlan; | ||
| use crate::cluster::{Cluster, WorkerPlanningParams}; | ||
| use crate::queryplanner::optimizations::distributed_partial_aggregate::{ | ||
| add_limit_to_workers, ensure_partition_merge, push_aggregate_to_workers, | ||
| replace_suboptimal_merge_sorts, | ||
| }; | ||
|
Check warning on line 14 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| use crate::queryplanner::optimizations::inline_aggregate_rewriter::replace_with_inline_aggregate; | ||
| use crate::queryplanner::check_memory::CheckMemoryExec; | ||
| use crate::queryplanner::inline_aggregate::{InlineAggregateExec, InlineAggregateMode}; | ||
| use crate::queryplanner::planning::CubeExtensionPlanner; | ||
| use crate::queryplanner::pretty_printers::{pp_phys_plan_ext, PPOptions}; | ||
| use crate::queryplanner::query_executor::ClusterSendExec; | ||
| use crate::queryplanner::rolling::RollingWindowPlanner; | ||
| use crate::queryplanner::trace_data_loaded::DataLoadedSize; | ||
| use crate::util::memory::MemoryHandler; | ||
|
|
@@ -25,7 +28,10 @@ | |
| use datafusion::execution::context::QueryPlanner; | ||
| use datafusion::execution::SessionState; | ||
| use datafusion::logical_expr::LogicalPlan; | ||
| use datafusion::physical_expr::LexOrdering; | ||
| use datafusion::physical_optimizer::PhysicalOptimizerRule; | ||
| use datafusion::physical_plan::projection::ProjectionExec; | ||
| use datafusion::physical_plan::sorts::sort::SortExec; | ||
| use datafusion::physical_plan::ExecutionPlan; | ||
| use datafusion::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; | ||
| use distributed_partial_aggregate::ensure_partition_merge_with_acceptable_parent; | ||
|
|
@@ -148,6 +154,10 @@ | |
| // Replace sorted AggregateExec with InlineAggregateExec for better performance | ||
| let p = rewrite_physical_plan(p, &mut |p| replace_with_inline_aggregate(p))?; | ||
|
|
||
| // Apply worker_sort_and_limit AFTER aggregate restructuring, so the SortExec | ||
| // wraps the aggregate output (not the raw scan input). | ||
| let p = rewrite_physical_plan(p, &mut |p| apply_worker_sort_and_limit(p))?; | ||
|
|
||
| Ok(p) | ||
| } | ||
|
|
||
|
|
@@ -177,10 +187,190 @@ | |
| "Rewrote physical plan by add_limit_to_workers:\n{}", | ||
| pp_phys_plan_ext(p.as_ref(), &PPOptions::show_nonmeta()) | ||
| ); | ||
| let p = push_sort_to_workers(p)?; | ||
| log::trace!( | ||
| "Rewrote physical plan by push_sort_to_workers:\n{}", | ||
| pp_phys_plan_ext(p.as_ref(), &PPOptions::show_nonmeta()) | ||
| ); | ||
| let p = rewrite_physical_plan(p, &mut |p| replace_suboptimal_merge_sorts(p))?; | ||
| log::trace!( | ||
| "Rewrote physical plan by replace_suboptimal_merge_sorts:\n{}", | ||
| pp_phys_plan_ext(p.as_ref(), &PPOptions::show_nonmeta()) | ||
| ); | ||
| Ok(p) | ||
| } | ||
|
|
||
| /// When the router plan has `SortExec(fetch=N)` sorting by GROUP BY columns, | ||
| /// and the worker uses `InlineAggregateExec` (streaming aggregate where groups don't overlap), | ||
| /// push a matching `SortExec(fetch=N)` to the worker. DataFusion's SortExec with fetch uses | ||
| /// a bounded heap, so this limits worker output to N rows with O(N) memory. | ||
| fn push_sort_to_workers( | ||
| p: Arc<dyn ExecutionPlan>, | ||
| ) -> Result<Arc<dyn ExecutionPlan>, DataFusionError> { | ||
| let Some(sort) = p.as_any().downcast_ref::<SortExec>() else { | ||
| return Ok(p); | ||
| }; | ||
| let Some(fetch) = sort.fetch() else { | ||
| return Ok(p); | ||
| }; | ||
|
|
||
| let sort_exprs = sort.expr().clone(); | ||
|
|
||
|
Check warning on line 218 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| // Walk down through Projection → InlineAggregate(Final) → ClusterSend | ||
| let sort_input = sort.input(); | ||
| let (below_proj, col_mapping) = if let Some(proj) = sort_input.as_any().downcast_ref::<ProjectionExec>() { | ||
| // Map sort column indices through the projection | ||
| let mapping: Vec<Option<usize>> = proj.expr().iter().map(|(expr, _name)| { | ||
| expr.as_any() | ||
| .downcast_ref::<datafusion::physical_plan::expressions::Column>() | ||
| .map(|c| c.index()) | ||
| }).collect(); | ||
| (proj.input().clone(), Some(mapping)) | ||
| } else { | ||
| (sort_input.clone(), None) | ||
| }; | ||
|
|
||
| let Some(inline_final) = below_proj.as_any().downcast_ref::<InlineAggregateExec>() else { | ||
| return Ok(p); | ||
| }; | ||
| if *inline_final.mode() != InlineAggregateMode::Final { | ||
| return Ok(p); | ||
| } | ||
| let group_key_len = inline_final.group_expr().expr().len(); | ||
|
Check warning on line 239 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
|
|
||
| // Translate sort expressions to pre-projection column indices | ||
| let translated_sort_exprs: Vec<_> = sort_exprs.iter().map(|se| { | ||
| if let Some(col) = se.expr.as_any().downcast_ref::<datafusion::physical_plan::expressions::Column>() { | ||
| let actual_idx = if let Some(ref mapping) = col_mapping { | ||
| mapping.get(col.index()).copied().flatten() | ||
| } else { | ||
| Some(col.index()) | ||
| }; | ||
| actual_idx.filter(|&idx| idx < group_key_len) | ||
| } else { | ||
| None | ||
| } | ||
| }).collect(); | ||
|
|
||
| // All sort columns must be GROUP BY columns | ||
| if translated_sort_exprs.iter().any(|x| x.is_none()) { | ||
| return Ok(p); | ||
| } | ||
|
|
||
| // Find ClusterSendExec below InlineAggregate(Final), possibly through CheckMemoryExec | ||
| let final_input = inline_final.input(); | ||
| let (cluster_send, through_check_memory) = | ||
| if let Some(cs) = final_input.as_any().downcast_ref::<ClusterSendExec>() { | ||
| (cs, false) | ||
| } else if let Some(cm) = final_input.as_any().downcast_ref::<CheckMemoryExec>() { | ||
| if let Some(cs) = cm.input.as_any().downcast_ref::<ClusterSendExec>() { | ||
| (cs, true) | ||
| } else { | ||
| return Ok(p); | ||
| } | ||
| } else { | ||
| return Ok(p); | ||
| }; | ||
|
|
||
| // Don't override if limit_and_reverse is already set | ||
| if cluster_send.limit_and_reverse.is_some() { | ||
| return Ok(p); | ||
| } | ||
|
|
||
| let worker_input = &cluster_send.input_for_optimizations; | ||
|
|
||
| // Verify the worker has InlineAggregateExec(Partial) - confirms groups don't overlap | ||
| let has_inline_partial = worker_input | ||
| .as_any() | ||
| .downcast_ref::<InlineAggregateExec>() | ||
| .map_or(false, |ia| *ia.mode() == InlineAggregateMode::Partial); | ||
| if !has_inline_partial { | ||
| return Ok(p); | ||
| } | ||
|
|
||
|
Check warning on line 290 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| // Build sort expressions for the worker (same column indices, same options) | ||
| let worker_schema = worker_input.schema(); | ||
| let worker_sort_exprs: Vec<_> = sort_exprs.iter().zip(translated_sort_exprs.iter()).map(|(se, &mapped_idx)| { | ||
| let idx = mapped_idx.unwrap(); | ||
| datafusion::physical_expr::PhysicalSortExpr { | ||
|
Comment on lines
+285
to
+295
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. Don't skip when Recommend: rather than |
||
| expr: Arc::new(datafusion::physical_plan::expressions::Column::new( | ||
| worker_schema.field(idx).name(), | ||
| idx, | ||
| )), | ||
| options: se.options, | ||
| } | ||
| }).collect(); | ||
|
|
||
| // Wrap the worker plan: SortExec(fetch=N) → InlinePartialAggregate | ||
| let new_worker_input: Arc<dyn ExecutionPlan> = Arc::new( | ||
| SortExec::new(LexOrdering::new(worker_sort_exprs), worker_input.clone()) | ||
| .with_fetch(Some(fetch)), | ||
| ); | ||
|
Check warning on line 308 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
|
|
||
| // Rebuild ClusterSendExec with the new worker input | ||
| let new_cluster_send: Arc<dyn ExecutionPlan> = Arc::new( | ||
| cluster_send.with_changed_schema(new_worker_input, cluster_send.required_input_ordering.clone()), | ||
| ); | ||
|
|
||
| // Re-wrap with CheckMemoryExec if it was present | ||
| let new_final_child: Arc<dyn ExecutionPlan> = if through_check_memory { | ||
| final_input.clone().with_new_children(vec![new_cluster_send])? | ||
|
Check warning on line 317 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| } else { | ||
| new_cluster_send | ||
| }; | ||
|
|
||
| // Rebuild InlineAggregate(Final) with new child | ||
| let new_inline_final: Arc<dyn ExecutionPlan> = | ||
| Arc::clone(&below_proj).with_new_children(vec![new_final_child])?; | ||
|
Check warning on line 324 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
|
|
||
| // Rebuild Projection if present | ||
| let new_sort_input = if sort_input.as_any().downcast_ref::<ProjectionExec>().is_some() { | ||
| sort_input.clone().with_new_children(vec![new_inline_final])? | ||
| } else { | ||
| new_inline_final | ||
| }; | ||
|
|
||
| // Rebuild SortExec with the new subtree | ||
| p.with_new_children(vec![new_sort_input]) | ||
| } | ||
|
|
||
| /// Apply worker_sort_and_limit on a WorkerExec by wrapping its child | ||
| /// (the partial aggregate) with SortExec(fetch=N). Must run AFTER | ||
| /// push_aggregate_to_workers and replace_with_inline_aggregate. | ||
| fn apply_worker_sort_and_limit( | ||
| p: Arc<dyn ExecutionPlan>, | ||
| ) -> Result<Arc<dyn ExecutionPlan>, DataFusionError> { | ||
| use crate::queryplanner::planning::WorkerExec; | ||
|
|
||
| let Some(w) = p.as_any().downcast_ref::<WorkerExec>() else { | ||
| return Ok(p); | ||
| }; | ||
| let Some((sort_cols, limit)) = w.worker_sort_and_limit.as_ref() else { | ||
| return Ok(p); | ||
| }; | ||
|
|
||
| let input = &w.input; | ||
| let schema = input.schema(); | ||
|
Check warning on line 353 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| let sort_exprs: Vec<_> = sort_cols | ||
| .iter() | ||
| .map(|(col_idx, asc, nulls_first)| { | ||
| datafusion::physical_expr::PhysicalSortExpr { | ||
| expr: Arc::new( | ||
| datafusion::physical_plan::expressions::Column::new( | ||
| schema.field(*col_idx).name(), | ||
| *col_idx, | ||
| ), | ||
| ), | ||
| options: datafusion::arrow::compute::SortOptions { | ||
| descending: !asc, | ||
| nulls_first: *nulls_first, | ||
| }, | ||
|
Check warning on line 367 in rust/cubestore/cubestore/src/queryplanner/optimizations/mod.rs
|
||
| } | ||
| }) | ||
| .collect(); | ||
| let sort_exec: Arc<dyn ExecutionPlan> = Arc::new( | ||
| SortExec::new(LexOrdering::new(sort_exprs), input.clone()) | ||
| .with_fetch(Some(*limit)), | ||
|
Comment on lines
+203
to
+373
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. Two parallel mechanisms for the same optimization — please clarify or consolidate. This PR introduces two distinct paths that both wrap the worker plan with
Workers re-plan from the serialized logical plan rather than receiving the router's physical plan, so path (1) doesn't actually reach a remote worker — it only affects local execution. Meanwhile (2) is the path that actually reaches remote workers via serialization. A few concrete concerns:
Consider either removing path (1) (since the logical-level field already covers the worker case end-to-end), or document very clearly which path fires in which deployment topology. Right now this is the single hardest piece of the PR to reason about. |
||
| ); | ||
| p.with_new_children(vec![sort_exec]) | ||
| } | ||
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.
InlineAggregateMode::Singleis added end-to-end but I don't see anything in this PR that actually constructs aSingle-modeAggregateExec— DataFusion only emitsAggregateMode::Singlefor a one-pass aggregate which Cube's distributed planner typically rewrites into Partial+Final. Is this dead code today, or is there a path I'm missing? If it's anticipatory, a test that exercises it (or at least a comment noting which planner path produces it) would prevent the new branches ininline_aggregate_stream.rsfrom going untested.