From 29e3814fb03b6880743c46caa3d79bde9f8378ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 17 Mar 2026 21:54:03 +0100 Subject: [PATCH 1/8] feat: implement GroupJoin optimization (Moerkotte & Neumann, VLDB 2011) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fuses AggregateExec(Partial) on top of HashJoinExec into a single GroupJoinExec operator, avoiding materialization of the large intermediate join result when GROUP BY columns come from the left side and aggregate arguments come from the right side. GroupJoinExec builds a hash table on the right (build) side by collecting all right partitions, then for each left (probe) row looks up matches and accumulates them directly into GroupsAccumulators via the existing GroupValues machinery. The optimizer rule (`GroupJoinOptimization`) matches: - AggregateExec → HashJoinExec - AggregateExec → ProjectionExec → HashJoinExec Preconditions: INNER join only, no join filter, non-empty GROUP BY, no GROUPING SETS / DISTINCT / ORDER BY / per-agg FILTER, group-by refs left side, agg args ref right side. TPC-H Q12 now uses GroupJoinExec. All sqllogictest plan expectations updated accordingly. All query result tests pass. Co-Authored-By: Claude Sonnet 4.6 --- .../physical-optimizer/src/group_join.rs | 366 ++++++++ datafusion/physical-optimizer/src/lib.rs | 1 + .../physical-optimizer/src/optimizer.rs | 5 + .../physical-plan/src/aggregates/mod.rs | 2 +- .../physical-plan/src/joins/group_join.rs | 843 ++++++++++++++++++ .../physical-plan/src/joins/hash_join/mod.rs | 4 +- .../src/joins/hash_join/stream.rs | 2 +- datafusion/physical-plan/src/joins/mod.rs | 1 + .../sqllogictest/test_files/explain.slt | 4 + .../sqllogictest/test_files/group_join.slt | 122 +++ datafusion/sqllogictest/test_files/joins.slt | 24 +- .../test_files/tpch/plans/q12.slt.part | 11 +- 12 files changed, 1362 insertions(+), 23 deletions(-) create mode 100644 datafusion/physical-optimizer/src/group_join.rs create mode 100644 datafusion/physical-plan/src/joins/group_join.rs create mode 100644 datafusion/sqllogictest/test_files/group_join.slt diff --git a/datafusion/physical-optimizer/src/group_join.rs b/datafusion/physical-optimizer/src/group_join.rs new file mode 100644 index 0000000000000..3c1b9a5f6bee4 --- /dev/null +++ b/datafusion/physical-optimizer/src/group_join.rs @@ -0,0 +1,366 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! GroupJoin optimizer rule. +//! +//! Implements the GroupJoin optimization from "Accelerating Queries with +//! Group-By and Join by Groupjoin" (Moerkotte & Neumann, VLDB 2011). + +use std::sync::Arc; + +use crate::PhysicalOptimizerRule; + +use datafusion_common::Result; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_expr::JoinType; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::joins::HashJoinExec; +use datafusion_physical_plan::joins::group_join::GroupJoinExec; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::{ExecutionPlan, PhysicalExpr}; + +/// Physical optimizer rule that fuses `AggregateExec` on top of `HashJoinExec` +/// into a single `GroupJoinExec` when the preconditions are met. +/// +/// Matches patterns: +/// - `AggregateExec → HashJoinExec` +/// - `AggregateExec → ProjectionExec → HashJoinExec` +#[derive(Debug, Default)] +pub struct GroupJoinOptimization {} + +impl GroupJoinOptimization { + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for GroupJoinOptimization { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(|plan: Arc| { + let Some(agg_exec) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + // Try matching AggregateExec → HashJoinExec directly + if let Some(hash_join) = + agg_exec.input().as_any().downcast_ref::() + && let Some(group_join) = + try_create_group_join(agg_exec, hash_join, None)? + { + return Ok(Transformed::yes(Arc::new(group_join))); + } + + // Try matching AggregateExec → ProjectionExec → HashJoinExec + if let Some(proj_exec) = + agg_exec.input().as_any().downcast_ref::() + && let Some(hash_join) = + proj_exec.input().as_any().downcast_ref::() + && let Some(group_join) = + try_create_group_join(agg_exec, hash_join, Some(proj_exec))? + { + return Ok(Transformed::yes(Arc::new(group_join))); + } + + Ok(Transformed::no(plan)) + }) + .data() + } + + fn name(&self) -> &str { + "GroupJoinOptimization" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// Builds a column index resolver. +/// +/// When there is no projection (or a direct HashJoinExec with its own projection): +/// - Resolves aggregate column indices to join schema indices. +/// +/// When there is a ProjectionExec between aggregate and join: +/// - First maps through the ProjectionExec's expressions, then through the +/// HashJoinExec's own projection (if any). +fn build_resolver( + hash_join: &HashJoinExec, + proj_exec: Option<&ProjectionExec>, +) -> Box Option> { + match proj_exec { + Some(proj) => { + // ProjectionExec maps aggregate columns to join output columns. + // Each projection expression should be a simple Column reference. + let proj_mapping: Vec> = proj + .expr() + .iter() + .map(|pe| { + pe.expr + .as_any() + .downcast_ref::() + .map(|col| col.index()) + }) + .collect(); + + // The HashJoinExec may also have its own projection + let join_proj = hash_join.projection.as_deref().map(|p| p.to_vec()); + + Box::new(move |agg_col_idx: usize| { + // Step 1: map through the ProjectionExec + let join_output_idx = proj_mapping.get(agg_col_idx)?.as_ref().copied()?; + + // Step 2: if the HashJoinExec has a projection, map through it + match &join_proj { + Some(jp) => jp.get(join_output_idx).copied(), + None => Some(join_output_idx), + } + }) + } + None => { + // No ProjectionExec — map directly through HashJoinExec's projection + let join_proj = hash_join.projection.as_deref().map(|p| p.to_vec()); + + Box::new(move |agg_col_idx: usize| match &join_proj { + Some(jp) => jp.get(agg_col_idx).copied(), + None => Some(agg_col_idx), + }) + } + } +} + +/// Attempts to create a `GroupJoinExec` from an `AggregateExec` on top of a `HashJoinExec`, +/// optionally with a `ProjectionExec` in between. +fn try_create_group_join( + agg_exec: &AggregateExec, + hash_join: &HashJoinExec, + proj_exec: Option<&ProjectionExec>, +) -> Result> { + // Only support INNER joins + if *hash_join.join_type() != JoinType::Inner { + return Ok(None); + } + + // No join filter + if hash_join.filter().is_some() { + return Ok(None); + } + + let group_by = agg_exec.group_expr(); + if group_by.has_grouping_set() { + return Ok(None); + } + + // Empty GROUP BY (global aggregate like COUNT(*)) is not supported: + // GroupValues::intern cannot determine row count from zero columns. + if group_by.expr().is_empty() { + return Ok(None); + } + + if agg_exec.filter_expr().iter().any(|f| f.is_some()) { + return Ok(None); + } + + if agg_exec.aggr_expr().iter().any(|e| e.is_distinct()) { + return Ok(None); + } + + if agg_exec + .aggr_expr() + .iter() + .any(|e| !e.order_bys().is_empty()) + { + return Ok(None); + } + + // The join output schema (before any projection) is [left_cols..., right_cols...] + let left_field_count = hash_join.left().schema().fields().len(); + + let resolve_index = build_resolver(hash_join, proj_exec); + + // Group-by expressions must all resolve to left-side join columns + for (expr, _alias) in group_by.expr() { + if !expr_columns_in_side(expr.as_ref(), &*resolve_index, 0, left_field_count) { + return Ok(None); + } + } + + // Aggregate arguments must all resolve to right-side join columns + for agg_expr in agg_exec.aggr_expr() { + for arg in agg_expr.expressions() { + if !expr_columns_in_side( + arg.as_ref(), + &*resolve_index, + left_field_count, + usize::MAX, + ) { + return Ok(None); + } + } + } + + // Remap group-by expressions to reference the left input schema directly + let remapped_group_by: Vec<_> = group_by + .expr() + .iter() + .map(|(expr, alias)| { + let remapped = remap_columns(expr, &*resolve_index, 0); + (remapped, alias.clone()) + }) + .collect(); + + // Remap aggregate arguments to reference the right input schema directly + let remapped_aggr_exprs: Option> = agg_exec + .aggr_expr() + .iter() + .map(|expr| { + let remapped_args: Vec> = expr + .expressions() + .iter() + .map(|a| remap_columns(a, &*resolve_index, left_field_count)) + .collect(); + expr.with_new_expressions(remapped_args, vec![]) + }) + .collect(); + + let Some(remapped_aggr_exprs) = remapped_aggr_exprs else { + return Ok(None); + }; + let remapped_aggr_exprs: Vec<_> = + remapped_aggr_exprs.into_iter().map(Arc::new).collect(); + + GroupJoinExec::try_new( + *agg_exec.mode(), + Arc::clone(hash_join.left()), + Arc::clone(hash_join.right()), + hash_join.on().to_vec(), + remapped_group_by, + remapped_aggr_exprs, + Arc::clone(&agg_exec.schema()), + ) + .map(Some) +} + +/// Check if all Column expressions in `expr`, after resolving, fall in [start, end). +fn expr_columns_in_side( + expr: &dyn PhysicalExpr, + resolve_index: &dyn Fn(usize) -> Option, + start: usize, + end: usize, +) -> bool { + let mut valid = true; + check_columns_resolved(expr, resolve_index, start, end, &mut valid); + valid +} + +fn check_columns_resolved( + expr: &dyn PhysicalExpr, + resolve_index: &dyn Fn(usize) -> Option, + start: usize, + end: usize, + valid: &mut bool, +) { + if !*valid { + return; + } + if let Some(col) = expr.as_any().downcast_ref::() + && !matches!(resolve_index(col.index()), Some(resolved) if resolved >= start && resolved < end) + { + *valid = false; + return; + } + for child in expr.children() { + check_columns_resolved(child.as_ref(), resolve_index, start, end, valid); + } +} + +/// Remap column expressions by resolving through the index resolver and subtracting `offset`. +fn remap_columns( + expr: &Arc, + resolve_index: &dyn Fn(usize) -> Option, + offset: usize, +) -> Arc { + Arc::clone(expr) + .transform(|e: Arc| { + if let Some(col) = e.as_any().downcast_ref::() { + let idx = col.index(); + let resolved = resolve_index(idx).unwrap_or(idx); + Ok(Transformed::yes(Arc::new(Column::new( + col.name(), + resolved - offset, + )))) + } else { + Ok(Transformed::no(e)) + } + }) + .unwrap() + .data +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_plan::PhysicalExpr; + + #[test] + fn test_expr_columns_in_side_without_projection() { + let col0: Arc = Arc::new(Column::new("a", 0)); + let col3: Arc = Arc::new(Column::new("b", 3)); + let identity = |idx: usize| -> Option { Some(idx) }; + + assert!(expr_columns_in_side(col0.as_ref(), &identity, 0, 4)); + assert!(expr_columns_in_side(col3.as_ref(), &identity, 0, 4)); + assert!(!expr_columns_in_side(col3.as_ref(), &identity, 0, 3)); + } + + #[test] + fn test_expr_columns_with_projection() { + // projection: [1, 3] means output col 0 → join col 1, output col 1 → join col 3 + let proj = [1usize, 3]; + let resolve = |idx: usize| -> Option { proj.get(idx).copied() }; + + let col0: Arc = Arc::new(Column::new("a", 0)); + let col1: Arc = Arc::new(Column::new("b", 1)); + + // col0 → join col 1, in left side [0, 2) + assert!(expr_columns_in_side(col0.as_ref(), &resolve, 0, 2)); + // col1 → join col 3, in right side [2, MAX) + assert!(expr_columns_in_side(col1.as_ref(), &resolve, 2, usize::MAX)); + assert!(!expr_columns_in_side(col1.as_ref(), &resolve, 0, 2)); + } + + #[test] + fn test_remap_columns() { + let proj = [1usize, 3]; + let resolve = |idx: usize| -> Option { proj.get(idx).copied() }; + + // col1 in aggregate → resolve to join col 3 → right input col 3-2=1 + let col1: Arc = Arc::new(Column::new("amount", 1)); + let remapped = remap_columns(&col1, &resolve, 2); + let col = remapped.as_any().downcast_ref::().unwrap(); + assert_eq!(col.index(), 1); + assert_eq!(col.name(), "amount"); + } +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index a328f43d22b2b..3a4f1951a2be7 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -31,6 +31,7 @@ pub mod enforce_distribution; pub mod enforce_sorting; pub mod ensure_coop; pub mod filter_pushdown; +pub mod group_join; pub mod join_selection; pub mod limit_pushdown; pub mod limit_pushdown_past_window; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index a51dc47999897..ca7e1c735c0b2 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -36,6 +36,7 @@ use crate::topk_aggregation::TopKAggregation; use crate::topk_repartition::TopKRepartition; use crate::update_aggr_exprs::OptimizeAggregateOrder; +use crate::group_join::GroupJoinOptimization; use crate::hash_join_buffering::HashJoinBuffering; use crate::limit_pushdown_past_window::LimitPushPastWindows; use crate::pushdown_sort::PushdownSort; @@ -105,6 +106,10 @@ impl PhysicalOptimizer { // as that rule may inject other operations in between the different AggregateExecs. // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), + // GroupJoin fuses AggregateExec + HashJoinExec into a single GroupJoinExec + // when group-by columns come from the left side and aggregates from the right side. + // Must run before EnforceDistribution as that may insert operators between them. + Arc::new(GroupJoinOptimization::new()), // The FilterPushdown rule tries to push down filters as far as it can. // For example, it will push down filtering from a `FilterExec` to `DataSourceExec`. // Note that this does not push down dynamic filters (such as those created by a `SortExec` operator in TopK mode), diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 42df1a8b07cd4..4474ebd24f1a1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -71,7 +71,7 @@ use topk::heap::is_supported_heap_type; pub mod group_values; mod no_grouping; pub mod order; -mod row_hash; +pub(crate) mod row_hash; mod topk; mod topk_stream; diff --git a/datafusion/physical-plan/src/joins/group_join.rs b/datafusion/physical-plan/src/joins/group_join.rs new file mode 100644 index 0000000000000..15d84c86b6a8d --- /dev/null +++ b/datafusion/physical-plan/src/joins/group_join.rs @@ -0,0 +1,843 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! GroupJoin execution plan. +//! +//! Implements a fused hash-join + aggregation operator based on +//! "Accelerating Queries with Group-By and Join by Groupjoin" +//! (Moerkotte & Neumann, VLDB 2011). +//! +//! The operator builds a hash table on the right (build) side, then +//! for each left (probe) row, looks up matching right rows and feeds +//! them directly into group-by accumulators—avoiding materialization of +//! the potentially large intermediate join result. + +use std::any::Any; +use std::fmt; +use std::sync::Arc; + +use arrow::array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; +use arrow::compute; +use arrow::datatypes::SchemaRef; +use datafusion_common::hash_utils::{RandomState, create_hashes}; +use datafusion_common::tree_node::TreeNodeRecursion; +use datafusion_common::utils::memory::get_record_batch_memory_size; +use datafusion_common::{NullEquality, Result}; +use datafusion_execution::TaskContext; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr_common::utils::evaluate_expressions_to_arrays; +use futures::StreamExt; + +use crate::aggregates::group_values::{GroupValues, new_group_values}; +use crate::aggregates::order::GroupOrdering; +use crate::aggregates::row_hash::create_group_accumulator; +use crate::aggregates::{ + AggregateMode, AggregateOutputMode, PhysicalGroupBy, evaluate_group_by, +}; +use crate::execution_plan::{CardinalityEffect, EmissionType, PlanProperties}; +use crate::joins::hash_join::stream::lookup_join_hashmap; +use crate::joins::join_hash_map::{JoinHashMapU32, JoinHashMapU64}; +use crate::joins::utils::{JoinHashMapType, update_hash}; +use crate::joins::{JoinOn, Map, MapOffset}; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; +use crate::stream::RecordBatchStreamAdapter; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PhysicalExpr, + SendableRecordBatchStream, +}; + +/// GroupJoinExec fuses a hash join with a subsequent aggregation. +/// +/// Instead of materializing the full join result and then grouping, +/// it builds a hash table on the right (build) side and, for each +/// left (probe) row, directly aggregates matching right-side values. +/// +/// This is beneficial when the join has high fan-out (many right rows +/// per left row) but the final result is much smaller after aggregation. +#[derive(Debug)] +pub struct GroupJoinExec { + /// Aggregation mode (Single or Partial) + mode: AggregateMode, + /// Left input (probe side) — group-by columns come from here + left: Arc, + /// Right input (build side) — aggregated columns come from here + right: Arc, + /// Equi-join keys: (left_expr, right_expr) + on: JoinOn, + /// Group-by expressions evaluated on the left (probe) input + group_by: PhysicalGroupBy, + /// Aggregate function expressions with arguments referencing the right (build) input + aggr_expr: Vec>, + /// Output schema (matches the original AggregateExec's output) + schema: SchemaRef, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// Cached plan properties + cache: Arc, +} + +impl GroupJoinExec { + pub fn try_new( + mode: AggregateMode, + left: Arc, + right: Arc, + on: JoinOn, + group_by_exprs: Vec<(PhysicalExprRef, String)>, + aggr_expr: Vec>, + schema: SchemaRef, + ) -> Result { + let group_by = PhysicalGroupBy::new_single(group_by_exprs); + let cache = Arc::new(Self::compute_properties( + left.properties(), + Arc::clone(&schema), + )); + + Ok(Self { + mode, + left, + right, + on, + group_by, + aggr_expr, + schema, + metrics: ExecutionPlanMetricsSet::new(), + cache, + }) + } + + fn compute_properties( + left_props: &PlanProperties, + schema: SchemaRef, + ) -> PlanProperties { + use datafusion_physical_expr::equivalence::EquivalenceProperties; + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + left_props.output_partitioning().clone(), + EmissionType::Final, + left_props.boundedness, + ) + } +} + +impl DisplayAs for GroupJoinExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let on = self + .on + .iter() + .map(|(l, r)| format!("({l}, {r})")) + .collect::>() + .join(", "); + let group_by: Vec<_> = self + .group_by + .expr() + .iter() + .map(|(e, alias)| format!("{e} as {alias}")) + .collect(); + let aggrs: Vec<_> = self + .aggr_expr + .iter() + .map(|e| e.name().to_string()) + .collect(); + write!( + f, + "GroupJoinExec: mode={:?}, on=[{}], group_by=[{}], aggr=[{}]", + self.mode, + on, + group_by.join(", "), + aggrs.join(", "), + ) + } + DisplayFormatType::TreeRender => { + let on = self + .on + .iter() + .map(|(l, r)| format!("({l} = {r})")) + .collect::>() + .join(", "); + writeln!(f, "on={on}")?; + let group_by: Vec<_> = self + .group_by + .expr() + .iter() + .map(|(e, alias)| format!("{e} as {alias}")) + .collect(); + writeln!(f, "group_by=[{}]", group_by.join(", "))?; + let aggrs: Vec<_> = self + .aggr_expr + .iter() + .map(|e| e.name().to_string()) + .collect(); + writeln!(f, "aggr=[{}]", aggrs.join(", "))?; + Ok(()) + } + } + } +} + +impl ExecutionPlan for GroupJoinExec { + fn name(&self) -> &'static str { + "GroupJoinExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &Arc { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.left, &self.right] + } + + fn maintains_input_order(&self) -> Vec { + vec![false, false] + } + + fn apply_expressions( + &self, + f: &mut dyn FnMut(&dyn PhysicalExpr) -> Result, + ) -> Result { + let mut tnr = TreeNodeRecursion::Continue; + for (left, right) in &self.on { + tnr = tnr.visit_sibling(|| f(left.as_ref()))?; + tnr = tnr.visit_sibling(|| f(right.as_ref()))?; + } + for (expr, _) in self.group_by.expr() { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } + for agg in &self.aggr_expr { + for arg in agg.expressions() { + tnr = tnr.visit_sibling(|| f(arg.as_ref()))?; + } + } + Ok(tnr) + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(GroupJoinExec::try_new( + self.mode, + Arc::clone(&children[0]), + Arc::clone(&children[1]), + self.on.clone(), + self.group_by.expr().to_vec(), + self.aggr_expr.clone(), + Arc::clone(&self.schema), + )?)) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let on_right: Vec = + self.on.iter().map(|on| Arc::clone(&on.1)).collect(); + let on_left: Vec = + self.on.iter().map(|on| Arc::clone(&on.0)).collect(); + + // Collect ALL right-side partitions so any left-side partitioning is safe, + // mirroring HashJoinExec(CollectLeft) which collects the full build side. + let right_part_count = self.right.output_partitioning().partition_count(); + let right_streams: Vec = (0..right_part_count) + .map(|p| self.right.execute(p, Arc::clone(&context))) + .collect::>()?; + + let left_stream = self.left.execute(partition, Arc::clone(&context))?; + + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + + let accumulators: Vec> = self + .aggr_expr + .iter() + .map(create_group_accumulator) + .collect::>()?; + + let aggregate_arguments: Vec>> = + self.aggr_expr.iter().map(|agg| agg.expressions()).collect(); + + let group_by = self.group_by.clone(); + let mode = self.mode; + let schema = Arc::clone(&self.schema); + let batch_size = context.session_config().batch_size(); + + let group_schema = group_by.group_schema(&self.left.schema())?; + let group_values = new_group_values(group_schema, &GroupOrdering::None)?; + + let reservation = + MemoryConsumer::new("GroupJoinExec").register(context.memory_pool()); + + let stream = GroupJoinStream { + schema: Arc::clone(&schema), + on_left, + on_right, + right_streams, + left_stream: Some(left_stream), + mode, + group_by, + aggregate_arguments, + accumulators, + group_values, + current_group_indices: Vec::new(), + batch_size, + baseline_metrics, + state: GroupJoinStreamState::BuildRight, + right_data: None, + reservation, + random_state: RandomState::with_seed(0), + hashes_buffer: Vec::new(), + }; + + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::unfold(stream, |mut stream| async move { + match stream.next_batch().await { + Ok(Some(batch)) => Some((Ok(batch), stream)), + Ok(None) => None, + Err(e) => Some((Err(e), stream)), + } + }), + ))) + } + + fn cardinality_effect(&self) -> CardinalityEffect { + CardinalityEffect::LowerEqual + } +} + +enum GroupJoinStreamState { + BuildRight, + ProbeAndAggregate, + Emit, + Done, +} + +struct RightData { + map: Arc, + batch: RecordBatch, + values: Vec, +} + +struct GroupJoinStream { + schema: SchemaRef, + on_left: Vec, + on_right: Vec, + + /// All right-side partition streams (collected exhaustively to build the hash table) + right_streams: Vec, + left_stream: Option, + + mode: AggregateMode, + group_by: PhysicalGroupBy, + aggregate_arguments: Vec>>, + accumulators: Vec>, + group_values: Box, + current_group_indices: Vec, + + batch_size: usize, + #[expect(dead_code)] + baseline_metrics: BaselineMetrics, + state: GroupJoinStreamState, + right_data: Option, + reservation: MemoryReservation, + random_state: RandomState, + hashes_buffer: Vec, +} + +impl GroupJoinStream { + async fn next_batch(&mut self) -> Result> { + loop { + match &self.state { + GroupJoinStreamState::BuildRight => { + self.build_right_side().await?; + self.state = GroupJoinStreamState::ProbeAndAggregate; + } + GroupJoinStreamState::ProbeAndAggregate => { + let left_stream = self.left_stream.as_mut().unwrap(); + match left_stream.next().await { + Some(Ok(batch)) => { + self.process_left_batch(&batch)?; + } + Some(Err(e)) => return Err(e), + None => { + self.state = GroupJoinStreamState::Emit; + } + } + } + GroupJoinStreamState::Emit => { + let result = self.emit_results()?; + self.state = GroupJoinStreamState::Done; + if result.num_rows() > 0 { + return Ok(Some(result)); + } + return Ok(None); + } + GroupJoinStreamState::Done => return Ok(None), + } + } + } + + async fn build_right_side(&mut self) -> Result<()> { + let right_streams = std::mem::take(&mut self.right_streams); + if right_streams.is_empty() { + return datafusion_common::internal_err!("Right streams already consumed"); + } + + let right_schema = right_streams[0].schema(); + let mut batches: Vec = Vec::new(); + let mut num_rows: usize = 0; + + // Drain all right-side partitions into a single batch collection. + for mut stream in right_streams { + while let Some(batch) = stream.next().await { + let batch = batch?; + let batch_mem = get_record_batch_memory_size(&batch); + self.reservation.try_grow(batch_mem)?; + num_rows += batch.num_rows(); + batches.push(batch); + } + } + + if num_rows == 0 { + self.right_data = Some(RightData { + map: Arc::new(Map::HashMap(Box::new(JoinHashMapU32::with_capacity(0)))), + batch: RecordBatch::new_empty(right_schema), + values: vec![], + }); + return Ok(()); + } + + let mut hashmap: Box = if num_rows > u32::MAX as usize { + Box::new(JoinHashMapU64::with_capacity(num_rows)) + } else { + Box::new(JoinHashMapU32::with_capacity(num_rows)) + }; + + let mut hash_buf = Vec::new(); + let mut offset = 0; + + for batch in batches.iter().rev() { + hash_buf.clear(); + hash_buf.resize(batch.num_rows(), 0); + update_hash( + &self.on_right, + batch, + &mut *hashmap, + offset, + &self.random_state, + &mut hash_buf, + 0, + true, + )?; + offset += batch.num_rows(); + } + + let batch = compute::concat_batches(&right_schema, batches.iter().rev())?; + let values = evaluate_expressions_to_arrays(&self.on_right, &batch)?; + + self.right_data = Some(RightData { + map: Arc::new(Map::HashMap(hashmap)), + batch, + values, + }); + + Ok(()) + } + + fn process_left_batch(&mut self, left_batch: &RecordBatch) -> Result<()> { + let right_data = self.right_data.as_ref().ok_or_else(|| { + datafusion_common::DataFusionError::Internal( + "Right data not built".to_string(), + ) + })?; + + if right_data.map.is_empty() { + return Ok(()); + } + + let left_key_values = evaluate_expressions_to_arrays(&self.on_left, left_batch)?; + + self.hashes_buffer.clear(); + self.hashes_buffer.resize(left_batch.num_rows(), 0); + create_hashes( + &left_key_values, + &self.random_state, + &mut self.hashes_buffer, + )?; + + let mut probe_indices_buf: Vec = Vec::new(); + let mut build_indices_buf: Vec = Vec::new(); + let mut offset: Option = Some((0, None)); + + // Pre-evaluate group-by and aggregate expressions once per batch + let group_by_values = evaluate_group_by(&self.group_by, left_batch)?; + + // Pre-evaluate aggregate arguments on the right batch + let right_arg_arrays: Vec> = self + .aggregate_arguments + .iter() + .map(|arg_exprs| { + arg_exprs + .iter() + .map(|expr| { + expr.evaluate(&right_data.batch) + .and_then(|v| v.into_array(right_data.batch.num_rows())) + }) + .collect::>>() + }) + .collect::>()?; + + while let Some(current_offset) = offset { + probe_indices_buf.clear(); + build_indices_buf.clear(); + + let (build_indices, probe_indices, next_offset) = + match right_data.map.as_ref() { + Map::HashMap(map) => lookup_join_hashmap( + map.as_ref(), + &right_data.values, + &left_key_values, + NullEquality::NullEqualsNothing, + &self.hashes_buffer, + self.batch_size, + current_offset, + &mut probe_indices_buf, + &mut build_indices_buf, + )?, + Map::ArrayMap(array_map) => { + let next = array_map.get_matched_indices_with_limit_offset( + &left_key_values, + self.batch_size, + current_offset, + &mut probe_indices_buf, + &mut build_indices_buf, + )?; + ( + UInt64Array::from(std::mem::take(&mut build_indices_buf)), + UInt32Array::from(std::mem::take(&mut probe_indices_buf)), + next, + ) + } + }; + + if build_indices.is_empty() { + offset = next_offset; + if offset.is_none() { + break; + } + continue; + } + + for group_values_arr in &group_by_values { + // Take group values at matched probe positions + let matched_group_values: Vec = group_values_arr + .iter() + .map(|arr| compute::take(arr.as_ref(), &probe_indices, None)) + .collect::>()?; + + self.group_values + .intern(&matched_group_values, &mut self.current_group_indices)?; + let total_num_groups = self.group_values.len(); + + // Feed matched right-side values to accumulators + for (acc_idx, acc) in self.accumulators.iter_mut().enumerate() { + let values: Vec = right_arg_arrays[acc_idx] + .iter() + .map(|arr| compute::take(arr.as_ref(), &build_indices, None)) + .collect::>()?; + + acc.update_batch( + &values, + &self.current_group_indices, + None, + total_num_groups, + )?; + } + } + + offset = next_offset; + } + + Ok(()) + } + + fn emit_results(&mut self) -> Result { + let group_columns = self.group_values.emit(EmitTo::All)?; + + let agg_columns: Vec = match self.mode.output_mode() { + AggregateOutputMode::Final => self + .accumulators + .iter_mut() + .map(|acc| acc.evaluate(EmitTo::All)) + .collect::>>()?, + AggregateOutputMode::Partial => self + .accumulators + .iter_mut() + .flat_map(|acc| match acc.state(EmitTo::All) { + Ok(states) => states.into_iter().map(Ok).collect::>(), + Err(e) => vec![Err(e)], + }) + .collect::>>()?, + }; + + let mut columns = group_columns; + columns.extend(agg_columns); + + if columns.is_empty() || columns[0].is_empty() { + return Ok(RecordBatch::new_empty(Arc::clone(&self.schema))); + } + + Ok(RecordBatch::try_new(Arc::clone(&self.schema), columns)?) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + + use arrow::array::{Int32Array, Int64Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_expr::expressions::Column; + + use crate::test::TestMemoryExec; + + fn build_task_context() -> Arc { + let runtime = RuntimeEnvBuilder::new().build_arc().unwrap(); + let config = SessionConfig::new().with_batch_size(4096); + Arc::new(TaskContext::new( + None, + "test_session".to_string(), + config, + Default::default(), + Default::default(), + Default::default(), + runtime, + )) + } + + #[tokio::test] + async fn test_group_join_basic() -> Result<()> { + // Left table: orders (order_id, customer_id) + let left_schema = Arc::new(Schema::new(vec![ + Field::new("order_id", DataType::Int32, false), + Field::new("customer_id", DataType::Int32, false), + ])); + let left_batch = RecordBatch::try_new( + Arc::clone(&left_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4])), + Arc::new(Int32Array::from(vec![10, 10, 20, 20])), + ], + )?; + + // Right table: line_items (order_id, amount) + let right_schema = Arc::new(Schema::new(vec![ + Field::new("order_id", DataType::Int32, false), + Field::new("amount", DataType::Int64, false), + ])); + let right_batch = RecordBatch::try_new( + Arc::clone(&right_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 1, 2, 3, 3, 3, 4])), + Arc::new(Int64Array::from(vec![100, 200, 300, 400, 500, 600, 700])), + ], + )?; + + let left_exec = Arc::new(TestMemoryExec::try_new( + &[vec![left_batch]], + Arc::clone(&left_schema), + None, + )?); + let right_exec = Arc::new(TestMemoryExec::try_new( + &[vec![right_batch]], + Arc::clone(&right_schema), + None, + )?); + + // GROUP BY customer_id, SUM(amount) + // Join on order_id + let on: JoinOn = vec![( + Arc::new(Column::new("order_id", 0)), + Arc::new(Column::new("order_id", 0)), + )]; + + let group_by_exprs = vec![( + Arc::new(Column::new("customer_id", 1)) as PhysicalExprRef, + "customer_id".to_string(), + )]; + + // SUM(amount) - amount is column 1 in the right schema + let sum_udf = datafusion_functions_aggregate::sum::sum_udaf(); + let sum_expr = Arc::new( + AggregateExprBuilder::new(sum_udf, vec![Arc::new(Column::new("amount", 1))]) + .schema(Arc::clone(&right_schema)) + .alias("sum_amount") + .build()?, + ); + + let output_schema = Arc::new(Schema::new(vec![ + Field::new("customer_id", DataType::Int32, false), + Field::new("sum_amount", DataType::Int64, true), + ])); + + let group_join = GroupJoinExec::try_new( + AggregateMode::Single, + left_exec, + right_exec, + on, + group_by_exprs, + vec![sum_expr], + output_schema, + )?; + + let context = build_task_context(); + let mut stream = group_join.execute(0, context)?; + + let mut results: Vec = Vec::new(); + while let Some(batch) = stream.next().await { + results.push(batch?); + } + + assert_eq!(results.len(), 1); + let result = &results[0]; + assert_eq!(result.num_rows(), 2); + + // Collect results into a map for order-independent comparison + let customer_ids = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let sums = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + + let mut result_map = std::collections::HashMap::new(); + for i in 0..result.num_rows() { + result_map.insert(customer_ids.value(i), sums.value(i)); + } + + // Customer 10 has orders 1, 2: + // order 1: amounts 100 + 200 = 300 + // order 2: amount 300 + // total = 600 + assert_eq!(result_map[&10], 600); + + // Customer 20 has orders 3, 4: + // order 3: amounts 400 + 500 + 600 = 1500 + // order 4: amount 700 + // total = 2200 + assert_eq!(result_map[&20], 2200); + + Ok(()) + } + + #[tokio::test] + async fn test_group_join_empty_right() -> Result<()> { + let left_schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, false), + Field::new("val", DataType::Int32, false), + ])); + let left_batch = RecordBatch::try_new( + Arc::clone(&left_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(Int32Array::from(vec![10, 20])), + ], + )?; + + let right_schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, false), + Field::new("amount", DataType::Int64, false), + ])); + // Empty right side + let right_batch = RecordBatch::new_empty(Arc::clone(&right_schema)); + + let left_exec = Arc::new(TestMemoryExec::try_new( + &[vec![left_batch]], + Arc::clone(&left_schema), + None, + )?); + let right_exec = Arc::new(TestMemoryExec::try_new( + &[vec![right_batch]], + Arc::clone(&right_schema), + None, + )?); + + let on: JoinOn = vec![( + Arc::new(Column::new("key", 0)), + Arc::new(Column::new("key", 0)), + )]; + + let group_by_exprs = vec![( + Arc::new(Column::new("val", 1)) as PhysicalExprRef, + "val".to_string(), + )]; + + let sum_udf = datafusion_functions_aggregate::sum::sum_udaf(); + let sum_expr = Arc::new( + AggregateExprBuilder::new(sum_udf, vec![Arc::new(Column::new("amount", 1))]) + .schema(Arc::clone(&right_schema)) + .alias("sum_amount") + .build()?, + ); + + let output_schema = Arc::new(Schema::new(vec![ + Field::new("val", DataType::Int32, false), + Field::new("sum_amount", DataType::Int64, true), + ])); + + let group_join = GroupJoinExec::try_new( + AggregateMode::Single, + left_exec, + right_exec, + on, + group_by_exprs, + vec![sum_expr], + output_schema, + )?; + + let context = build_task_context(); + let mut stream = group_join.execute(0, context)?; + + let mut results: Vec = Vec::new(); + while let Some(batch) = stream.next().await { + results.push(batch?); + } + + // INNER join with empty right → no results + assert!(results.is_empty()); + + Ok(()) + } +} diff --git a/datafusion/physical-plan/src/joins/hash_join/mod.rs b/datafusion/physical-plan/src/joins/hash_join/mod.rs index b915802ea4015..d97451b3c0767 100644 --- a/datafusion/physical-plan/src/joins/hash_join/mod.rs +++ b/datafusion/physical-plan/src/joins/hash_join/mod.rs @@ -20,8 +20,8 @@ pub use exec::{HashJoinExec, HashJoinExecBuilder}; pub use partitioned_hash_eval::{HashExpr, HashTableLookupExpr, SeededRandomState}; -mod exec; +pub(crate) mod exec; mod inlist_builder; mod partitioned_hash_eval; mod shared_bounds; -mod stream; +pub(crate) mod stream; diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index ab630920184d3..5165ccb720fbb 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -283,7 +283,7 @@ impl RecordBatchStream for HashJoinStream { /// Probe indices: 3, 3, 4, 5 /// ``` #[expect(clippy::too_many_arguments)] -pub(super) fn lookup_join_hashmap( +pub(crate) fn lookup_join_hashmap( build_hashmap: &dyn JoinHashMapType, build_side_values: &[ArrayRef], probe_side_values: &[ArrayRef], diff --git a/datafusion/physical-plan/src/joins/mod.rs b/datafusion/physical-plan/src/joins/mod.rs index 2cdfa1e6ac020..0d99d60bcfcae 100644 --- a/datafusion/physical-plan/src/joins/mod.rs +++ b/datafusion/physical-plan/src/joins/mod.rs @@ -31,6 +31,7 @@ pub use sort_merge_join::SortMergeJoinExec; pub use symmetric_hash_join::SymmetricHashJoinExec; pub mod chain; mod cross_join; +pub mod group_join; mod hash_join; mod nested_loop_join; mod piecewise_merge_join; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 9916892058569..ef2a6941d2bb1 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -234,6 +234,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE @@ -314,6 +315,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE @@ -360,6 +362,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE @@ -605,6 +608,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE diff --git a/datafusion/sqllogictest/test_files/group_join.slt b/datafusion/sqllogictest/test_files/group_join.slt new file mode 100644 index 0000000000000..7ae72b9284e72 --- /dev/null +++ b/datafusion/sqllogictest/test_files/group_join.slt @@ -0,0 +1,122 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# Tests for GroupJoin optimization +# (fusing GROUP BY + aggregation with INNER JOIN) + +# Setup test tables +statement ok +CREATE TABLE orders (order_id INT, customer_id INT) AS VALUES + (1, 10), + (2, 10), + (3, 20), + (4, 20); + +statement ok +CREATE TABLE line_items (order_id INT, amount BIGINT) AS VALUES + (1, 100), + (1, 200), + (2, 300), + (3, 400), + (3, 500), + (3, 600), + (4, 700); + +# Verify GroupJoinExec appears in the physical plan +query TT +EXPLAIN SELECT o.customer_id, SUM(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +logical_plan +01)Aggregate: groupBy=[[o.customer_id]], aggr=[[sum(li.amount)]] +02)--Projection: o.customer_id, li.amount +03)----Inner Join: o.order_id = li.order_id +04)------SubqueryAlias: o +05)--------TableScan: orders projection=[order_id, customer_id] +06)------SubqueryAlias: li +07)--------TableScan: line_items projection=[order_id, amount] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[customer_id@0 as customer_id], aggr=[sum(li.amount)] +02)--RepartitionExec: partitioning=Hash([customer_id@0], 4), input_partitions=1 +03)----GroupJoinExec: mode=Partial, on=[(order_id@0, order_id@0)], group_by=[customer_id@1 as customer_id], aggr=[sum(li.amount)] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------DataSourceExec: partitions=1, partition_sizes=[1] + +# Verify correctness: customer 10 = 100+200+300 = 600, customer 20 = 400+500+600+700 = 2200 +query II rowsort +SELECT o.customer_id, SUM(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +10 600 +20 2200 + +# GroupJoin with COUNT +query II rowsort +SELECT o.customer_id, COUNT(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +10 3 +20 4 + +# GroupJoin with multiple aggregates +query IIRI rowsort +SELECT o.customer_id, SUM(li.amount), AVG(li.amount), COUNT(*) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +10 600 200 3 +20 2200 550 4 + +# GroupJoin with GROUP BY = join key (simplest case) +query II rowsort +SELECT o.order_id, SUM(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.order_id; +---- +1 300 +2 300 +3 1500 +4 700 + +# No matching rows scenario +statement ok +CREATE TABLE empty_items (order_id INT, amount BIGINT) AS VALUES (99, 0); + +query II rowsort +SELECT o.customer_id, SUM(ei.amount) +FROM orders o +JOIN empty_items ei ON o.order_id = ei.order_id +GROUP BY o.customer_id; +---- + +# Cleanup +statement ok +DROP TABLE orders; + +statement ok +DROP TABLE line_items; + +statement ok +DROP TABLE empty_items; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 136a68573562a..d1e729cfd2b17 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1340,11 +1340,11 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[] 02)--RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +03)----GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as t1_id], aggr=[] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)--------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1395,11 +1395,11 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +04)------GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1422,11 +1422,11 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +07)------------GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as alias1], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index b152fde02f060..d5f82737baecd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -64,10 +64,7 @@ physical_plan 03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 05)--------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] -08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -09)----------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false -11)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false +06)----------GroupJoinExec: mode=Partial, on=[(l_orderkey@0, o_orderkey@0)], group_by=[l_shipmode@1 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +07)------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] +08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false +09)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false From 664954d975f52566bed1c9e3b4cf1ab7695b3c8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 17 Mar 2026 21:54:03 +0100 Subject: [PATCH 2/8] Extra --- .../physical-optimizer/src/group_join.rs | 388 ++++++++ datafusion/physical-optimizer/src/lib.rs | 1 + .../physical-optimizer/src/optimizer.rs | 5 + .../physical-plan/src/aggregates/mod.rs | 2 +- .../physical-plan/src/joins/group_join.rs | 843 ++++++++++++++++++ .../physical-plan/src/joins/hash_join/mod.rs | 4 +- .../src/joins/hash_join/stream.rs | 2 +- datafusion/physical-plan/src/joins/mod.rs | 1 + .../sqllogictest/test_files/explain.slt | 4 + .../sqllogictest/test_files/group_join.slt | 122 +++ datafusion/sqllogictest/test_files/joins.slt | 24 +- .../test_files/tpch/plans/q12.slt.part | 11 +- 12 files changed, 1384 insertions(+), 23 deletions(-) create mode 100644 datafusion/physical-optimizer/src/group_join.rs create mode 100644 datafusion/physical-plan/src/joins/group_join.rs create mode 100644 datafusion/sqllogictest/test_files/group_join.slt diff --git a/datafusion/physical-optimizer/src/group_join.rs b/datafusion/physical-optimizer/src/group_join.rs new file mode 100644 index 0000000000000..056a22c81652a --- /dev/null +++ b/datafusion/physical-optimizer/src/group_join.rs @@ -0,0 +1,388 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! GroupJoin optimizer rule. +//! +//! Implements the GroupJoin optimization from "Accelerating Queries with +//! Group-By and Join by Groupjoin" (Moerkotte & Neumann, VLDB 2011). + +use std::collections::HashMap; +use std::sync::Arc; + +use crate::PhysicalOptimizerRule; + +use datafusion_common::Result; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_expr::JoinType; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::joins::HashJoinExec; +use datafusion_physical_plan::joins::group_join::GroupJoinExec; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::{ExecutionPlan, PhysicalExpr}; + +/// Physical optimizer rule that fuses `AggregateExec` on top of `HashJoinExec` +/// into a single `GroupJoinExec` when the preconditions are met. +/// +/// Matches patterns: +/// - `AggregateExec → HashJoinExec` +/// - `AggregateExec → ProjectionExec → HashJoinExec` +#[derive(Debug, Default)] +pub struct GroupJoinOptimization {} + +impl GroupJoinOptimization { + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for GroupJoinOptimization { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(|plan: Arc| { + let Some(agg_exec) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + // Try matching AggregateExec → HashJoinExec directly + if let Some(hash_join) = + agg_exec.input().as_any().downcast_ref::() + && let Some(group_join) = + try_create_group_join(agg_exec, hash_join, None)? + { + return Ok(Transformed::yes(Arc::new(group_join))); + } + + // Try matching AggregateExec → ProjectionExec → HashJoinExec + if let Some(proj_exec) = + agg_exec.input().as_any().downcast_ref::() + && let Some(hash_join) = + proj_exec.input().as_any().downcast_ref::() + && let Some(group_join) = + try_create_group_join(agg_exec, hash_join, Some(proj_exec))? + { + return Ok(Transformed::yes(Arc::new(group_join))); + } + + Ok(Transformed::no(plan)) + }) + .data() + } + + fn name(&self) -> &str { + "GroupJoinOptimization" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// Builds a column index resolver. +/// +/// When there is no projection (or a direct HashJoinExec with its own projection): +/// - Resolves aggregate column indices to join schema indices. +/// +/// When there is a ProjectionExec between aggregate and join: +/// - First maps through the ProjectionExec's expressions, then through the +/// HashJoinExec's own projection (if any). +fn build_resolver( + hash_join: &HashJoinExec, + proj_exec: Option<&ProjectionExec>, +) -> Box Option> { + match proj_exec { + Some(proj) => { + // ProjectionExec maps aggregate columns to join output columns. + // Each projection expression should be a simple Column reference. + let proj_mapping: Vec> = proj + .expr() + .iter() + .map(|pe| { + pe.expr + .as_any() + .downcast_ref::() + .map(|col| col.index()) + }) + .collect(); + + // The HashJoinExec may also have its own projection + let join_proj = hash_join.projection.as_deref().map(|p| p.to_vec()); + + Box::new(move |agg_col_idx: usize| { + // Step 1: map through the ProjectionExec + let join_output_idx = proj_mapping.get(agg_col_idx)?.as_ref().copied()?; + + // Step 2: if the HashJoinExec has a projection, map through it + match &join_proj { + Some(jp) => jp.get(join_output_idx).copied(), + None => Some(join_output_idx), + } + }) + } + None => { + // No ProjectionExec — map directly through HashJoinExec's projection + let join_proj = hash_join.projection.as_deref().map(|p| p.to_vec()); + + Box::new(move |agg_col_idx: usize| match &join_proj { + Some(jp) => jp.get(agg_col_idx).copied(), + None => Some(agg_col_idx), + }) + } + } +} + +/// Attempts to create a `GroupJoinExec` from an `AggregateExec` on top of a `HashJoinExec`, +/// optionally with a `ProjectionExec` in between. +fn try_create_group_join( + agg_exec: &AggregateExec, + hash_join: &HashJoinExec, + proj_exec: Option<&ProjectionExec>, +) -> Result> { + // Only support INNER joins + if *hash_join.join_type() != JoinType::Inner { + return Ok(None); + } + + // No join filter + if hash_join.filter().is_some() { + return Ok(None); + } + + let group_by = agg_exec.group_expr(); + if group_by.has_grouping_set() { + return Ok(None); + } + + // Empty GROUP BY (global aggregate like COUNT(*)) is not supported: + // GroupValues::intern cannot determine row count from zero columns. + if group_by.expr().is_empty() { + return Ok(None); + } + + if agg_exec.filter_expr().iter().any(|f| f.is_some()) { + return Ok(None); + } + + if agg_exec.aggr_expr().iter().any(|e| e.is_distinct()) { + return Ok(None); + } + + if agg_exec + .aggr_expr() + .iter() + .any(|e| !e.order_bys().is_empty()) + { + return Ok(None); + } + + // The join output schema (before any projection) is [left_cols..., right_cols...] + let left_field_count = hash_join.left().schema().fields().len(); + + let resolve_index = build_resolver(hash_join, proj_exec); + + // Build a substitution map: right-side join-key (raw join schema index) → left-side key index. + // When a GROUP BY column references a right join key `right_j`, and there is an ON condition + // `left_i = right_j`, we can substitute it with `left_i` (they are equal by the join predicate). + // This is Section 3.2 of Moerkotte & Neumann VLDB 2011. + let key_subst: HashMap = hash_join + .on() + .iter() + .filter_map(|(l_expr, r_expr)| { + let l_col = l_expr.as_any().downcast_ref::()?; + let r_col = r_expr.as_any().downcast_ref::()?; + Some((left_field_count + r_col.index(), l_col.index())) + }) + .collect(); + + // Resolver for group-by: after resolving through projections, substitute any right join key + // with its equivalent left join key. + let resolve_gby = |idx: usize| -> Option { + let resolved = resolve_index(idx)?; + Some(key_subst.get(&resolved).copied().unwrap_or(resolved)) + }; + + // Group-by expressions must all resolve to left-side join columns (after substitution) + for (expr, _alias) in group_by.expr() { + if !expr_columns_in_side(expr.as_ref(), &resolve_gby, 0, left_field_count) { + return Ok(None); + } + } + + // Aggregate arguments must all resolve to right-side join columns + for agg_expr in agg_exec.aggr_expr() { + for arg in agg_expr.expressions() { + if !expr_columns_in_side( + arg.as_ref(), + &*resolve_index, + left_field_count, + usize::MAX, + ) { + return Ok(None); + } + } + } + + // Remap group-by expressions to reference the left input schema directly (with substitution) + let remapped_group_by: Vec<_> = group_by + .expr() + .iter() + .map(|(expr, alias)| { + let remapped = remap_columns(expr, &resolve_gby, 0); + (remapped, alias.clone()) + }) + .collect(); + + // Remap aggregate arguments to reference the right input schema directly + let remapped_aggr_exprs: Option> = agg_exec + .aggr_expr() + .iter() + .map(|expr| { + let remapped_args: Vec> = expr + .expressions() + .iter() + .map(|a| remap_columns(a, &*resolve_index, left_field_count)) + .collect(); + expr.with_new_expressions(remapped_args, vec![]) + }) + .collect(); + + let Some(remapped_aggr_exprs) = remapped_aggr_exprs else { + return Ok(None); + }; + let remapped_aggr_exprs: Vec<_> = + remapped_aggr_exprs.into_iter().map(Arc::new).collect(); + + GroupJoinExec::try_new( + *agg_exec.mode(), + Arc::clone(hash_join.left()), + Arc::clone(hash_join.right()), + hash_join.on().to_vec(), + remapped_group_by, + remapped_aggr_exprs, + Arc::clone(&agg_exec.schema()), + ) + .map(Some) +} + +/// Check if all Column expressions in `expr`, after resolving, fall in [start, end). +fn expr_columns_in_side( + expr: &dyn PhysicalExpr, + resolve_index: &dyn Fn(usize) -> Option, + start: usize, + end: usize, +) -> bool { + let mut valid = true; + check_columns_resolved(expr, resolve_index, start, end, &mut valid); + valid +} + +fn check_columns_resolved( + expr: &dyn PhysicalExpr, + resolve_index: &dyn Fn(usize) -> Option, + start: usize, + end: usize, + valid: &mut bool, +) { + if !*valid { + return; + } + if let Some(col) = expr.as_any().downcast_ref::() + && !matches!(resolve_index(col.index()), Some(resolved) if resolved >= start && resolved < end) + { + *valid = false; + return; + } + for child in expr.children() { + check_columns_resolved(child.as_ref(), resolve_index, start, end, valid); + } +} + +/// Remap column expressions by resolving through the index resolver and subtracting `offset`. +fn remap_columns( + expr: &Arc, + resolve_index: &dyn Fn(usize) -> Option, + offset: usize, +) -> Arc { + Arc::clone(expr) + .transform(|e: Arc| { + if let Some(col) = e.as_any().downcast_ref::() { + let idx = col.index(); + let resolved = resolve_index(idx).unwrap_or(idx); + Ok(Transformed::yes(Arc::new(Column::new( + col.name(), + resolved - offset, + )))) + } else { + Ok(Transformed::no(e)) + } + }) + .unwrap() + .data +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_plan::PhysicalExpr; + + #[test] + fn test_expr_columns_in_side_without_projection() { + let col0: Arc = Arc::new(Column::new("a", 0)); + let col3: Arc = Arc::new(Column::new("b", 3)); + let identity = |idx: usize| -> Option { Some(idx) }; + + assert!(expr_columns_in_side(col0.as_ref(), &identity, 0, 4)); + assert!(expr_columns_in_side(col3.as_ref(), &identity, 0, 4)); + assert!(!expr_columns_in_side(col3.as_ref(), &identity, 0, 3)); + } + + #[test] + fn test_expr_columns_with_projection() { + // projection: [1, 3] means output col 0 → join col 1, output col 1 → join col 3 + let proj = [1usize, 3]; + let resolve = |idx: usize| -> Option { proj.get(idx).copied() }; + + let col0: Arc = Arc::new(Column::new("a", 0)); + let col1: Arc = Arc::new(Column::new("b", 1)); + + // col0 → join col 1, in left side [0, 2) + assert!(expr_columns_in_side(col0.as_ref(), &resolve, 0, 2)); + // col1 → join col 3, in right side [2, MAX) + assert!(expr_columns_in_side(col1.as_ref(), &resolve, 2, usize::MAX)); + assert!(!expr_columns_in_side(col1.as_ref(), &resolve, 0, 2)); + } + + #[test] + fn test_remap_columns() { + let proj = [1usize, 3]; + let resolve = |idx: usize| -> Option { proj.get(idx).copied() }; + + // col1 in aggregate → resolve to join col 3 → right input col 3-2=1 + let col1: Arc = Arc::new(Column::new("amount", 1)); + let remapped = remap_columns(&col1, &resolve, 2); + let col = remapped.as_any().downcast_ref::().unwrap(); + assert_eq!(col.index(), 1); + assert_eq!(col.name(), "amount"); + } +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index a328f43d22b2b..3a4f1951a2be7 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -31,6 +31,7 @@ pub mod enforce_distribution; pub mod enforce_sorting; pub mod ensure_coop; pub mod filter_pushdown; +pub mod group_join; pub mod join_selection; pub mod limit_pushdown; pub mod limit_pushdown_past_window; diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index a51dc47999897..ca7e1c735c0b2 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -36,6 +36,7 @@ use crate::topk_aggregation::TopKAggregation; use crate::topk_repartition::TopKRepartition; use crate::update_aggr_exprs::OptimizeAggregateOrder; +use crate::group_join::GroupJoinOptimization; use crate::hash_join_buffering::HashJoinBuffering; use crate::limit_pushdown_past_window::LimitPushPastWindows; use crate::pushdown_sort::PushdownSort; @@ -105,6 +106,10 @@ impl PhysicalOptimizer { // as that rule may inject other operations in between the different AggregateExecs. // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), + // GroupJoin fuses AggregateExec + HashJoinExec into a single GroupJoinExec + // when group-by columns come from the left side and aggregates from the right side. + // Must run before EnforceDistribution as that may insert operators between them. + Arc::new(GroupJoinOptimization::new()), // The FilterPushdown rule tries to push down filters as far as it can. // For example, it will push down filtering from a `FilterExec` to `DataSourceExec`. // Note that this does not push down dynamic filters (such as those created by a `SortExec` operator in TopK mode), diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 42df1a8b07cd4..4474ebd24f1a1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -71,7 +71,7 @@ use topk::heap::is_supported_heap_type; pub mod group_values; mod no_grouping; pub mod order; -mod row_hash; +pub(crate) mod row_hash; mod topk; mod topk_stream; diff --git a/datafusion/physical-plan/src/joins/group_join.rs b/datafusion/physical-plan/src/joins/group_join.rs new file mode 100644 index 0000000000000..15d84c86b6a8d --- /dev/null +++ b/datafusion/physical-plan/src/joins/group_join.rs @@ -0,0 +1,843 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! GroupJoin execution plan. +//! +//! Implements a fused hash-join + aggregation operator based on +//! "Accelerating Queries with Group-By and Join by Groupjoin" +//! (Moerkotte & Neumann, VLDB 2011). +//! +//! The operator builds a hash table on the right (build) side, then +//! for each left (probe) row, looks up matching right rows and feeds +//! them directly into group-by accumulators—avoiding materialization of +//! the potentially large intermediate join result. + +use std::any::Any; +use std::fmt; +use std::sync::Arc; + +use arrow::array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; +use arrow::compute; +use arrow::datatypes::SchemaRef; +use datafusion_common::hash_utils::{RandomState, create_hashes}; +use datafusion_common::tree_node::TreeNodeRecursion; +use datafusion_common::utils::memory::get_record_batch_memory_size; +use datafusion_common::{NullEquality, Result}; +use datafusion_execution::TaskContext; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr_common::utils::evaluate_expressions_to_arrays; +use futures::StreamExt; + +use crate::aggregates::group_values::{GroupValues, new_group_values}; +use crate::aggregates::order::GroupOrdering; +use crate::aggregates::row_hash::create_group_accumulator; +use crate::aggregates::{ + AggregateMode, AggregateOutputMode, PhysicalGroupBy, evaluate_group_by, +}; +use crate::execution_plan::{CardinalityEffect, EmissionType, PlanProperties}; +use crate::joins::hash_join::stream::lookup_join_hashmap; +use crate::joins::join_hash_map::{JoinHashMapU32, JoinHashMapU64}; +use crate::joins::utils::{JoinHashMapType, update_hash}; +use crate::joins::{JoinOn, Map, MapOffset}; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; +use crate::stream::RecordBatchStreamAdapter; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PhysicalExpr, + SendableRecordBatchStream, +}; + +/// GroupJoinExec fuses a hash join with a subsequent aggregation. +/// +/// Instead of materializing the full join result and then grouping, +/// it builds a hash table on the right (build) side and, for each +/// left (probe) row, directly aggregates matching right-side values. +/// +/// This is beneficial when the join has high fan-out (many right rows +/// per left row) but the final result is much smaller after aggregation. +#[derive(Debug)] +pub struct GroupJoinExec { + /// Aggregation mode (Single or Partial) + mode: AggregateMode, + /// Left input (probe side) — group-by columns come from here + left: Arc, + /// Right input (build side) — aggregated columns come from here + right: Arc, + /// Equi-join keys: (left_expr, right_expr) + on: JoinOn, + /// Group-by expressions evaluated on the left (probe) input + group_by: PhysicalGroupBy, + /// Aggregate function expressions with arguments referencing the right (build) input + aggr_expr: Vec>, + /// Output schema (matches the original AggregateExec's output) + schema: SchemaRef, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// Cached plan properties + cache: Arc, +} + +impl GroupJoinExec { + pub fn try_new( + mode: AggregateMode, + left: Arc, + right: Arc, + on: JoinOn, + group_by_exprs: Vec<(PhysicalExprRef, String)>, + aggr_expr: Vec>, + schema: SchemaRef, + ) -> Result { + let group_by = PhysicalGroupBy::new_single(group_by_exprs); + let cache = Arc::new(Self::compute_properties( + left.properties(), + Arc::clone(&schema), + )); + + Ok(Self { + mode, + left, + right, + on, + group_by, + aggr_expr, + schema, + metrics: ExecutionPlanMetricsSet::new(), + cache, + }) + } + + fn compute_properties( + left_props: &PlanProperties, + schema: SchemaRef, + ) -> PlanProperties { + use datafusion_physical_expr::equivalence::EquivalenceProperties; + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + left_props.output_partitioning().clone(), + EmissionType::Final, + left_props.boundedness, + ) + } +} + +impl DisplayAs for GroupJoinExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let on = self + .on + .iter() + .map(|(l, r)| format!("({l}, {r})")) + .collect::>() + .join(", "); + let group_by: Vec<_> = self + .group_by + .expr() + .iter() + .map(|(e, alias)| format!("{e} as {alias}")) + .collect(); + let aggrs: Vec<_> = self + .aggr_expr + .iter() + .map(|e| e.name().to_string()) + .collect(); + write!( + f, + "GroupJoinExec: mode={:?}, on=[{}], group_by=[{}], aggr=[{}]", + self.mode, + on, + group_by.join(", "), + aggrs.join(", "), + ) + } + DisplayFormatType::TreeRender => { + let on = self + .on + .iter() + .map(|(l, r)| format!("({l} = {r})")) + .collect::>() + .join(", "); + writeln!(f, "on={on}")?; + let group_by: Vec<_> = self + .group_by + .expr() + .iter() + .map(|(e, alias)| format!("{e} as {alias}")) + .collect(); + writeln!(f, "group_by=[{}]", group_by.join(", "))?; + let aggrs: Vec<_> = self + .aggr_expr + .iter() + .map(|e| e.name().to_string()) + .collect(); + writeln!(f, "aggr=[{}]", aggrs.join(", "))?; + Ok(()) + } + } + } +} + +impl ExecutionPlan for GroupJoinExec { + fn name(&self) -> &'static str { + "GroupJoinExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &Arc { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.left, &self.right] + } + + fn maintains_input_order(&self) -> Vec { + vec![false, false] + } + + fn apply_expressions( + &self, + f: &mut dyn FnMut(&dyn PhysicalExpr) -> Result, + ) -> Result { + let mut tnr = TreeNodeRecursion::Continue; + for (left, right) in &self.on { + tnr = tnr.visit_sibling(|| f(left.as_ref()))?; + tnr = tnr.visit_sibling(|| f(right.as_ref()))?; + } + for (expr, _) in self.group_by.expr() { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } + for agg in &self.aggr_expr { + for arg in agg.expressions() { + tnr = tnr.visit_sibling(|| f(arg.as_ref()))?; + } + } + Ok(tnr) + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(GroupJoinExec::try_new( + self.mode, + Arc::clone(&children[0]), + Arc::clone(&children[1]), + self.on.clone(), + self.group_by.expr().to_vec(), + self.aggr_expr.clone(), + Arc::clone(&self.schema), + )?)) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let on_right: Vec = + self.on.iter().map(|on| Arc::clone(&on.1)).collect(); + let on_left: Vec = + self.on.iter().map(|on| Arc::clone(&on.0)).collect(); + + // Collect ALL right-side partitions so any left-side partitioning is safe, + // mirroring HashJoinExec(CollectLeft) which collects the full build side. + let right_part_count = self.right.output_partitioning().partition_count(); + let right_streams: Vec = (0..right_part_count) + .map(|p| self.right.execute(p, Arc::clone(&context))) + .collect::>()?; + + let left_stream = self.left.execute(partition, Arc::clone(&context))?; + + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); + + let accumulators: Vec> = self + .aggr_expr + .iter() + .map(create_group_accumulator) + .collect::>()?; + + let aggregate_arguments: Vec>> = + self.aggr_expr.iter().map(|agg| agg.expressions()).collect(); + + let group_by = self.group_by.clone(); + let mode = self.mode; + let schema = Arc::clone(&self.schema); + let batch_size = context.session_config().batch_size(); + + let group_schema = group_by.group_schema(&self.left.schema())?; + let group_values = new_group_values(group_schema, &GroupOrdering::None)?; + + let reservation = + MemoryConsumer::new("GroupJoinExec").register(context.memory_pool()); + + let stream = GroupJoinStream { + schema: Arc::clone(&schema), + on_left, + on_right, + right_streams, + left_stream: Some(left_stream), + mode, + group_by, + aggregate_arguments, + accumulators, + group_values, + current_group_indices: Vec::new(), + batch_size, + baseline_metrics, + state: GroupJoinStreamState::BuildRight, + right_data: None, + reservation, + random_state: RandomState::with_seed(0), + hashes_buffer: Vec::new(), + }; + + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::unfold(stream, |mut stream| async move { + match stream.next_batch().await { + Ok(Some(batch)) => Some((Ok(batch), stream)), + Ok(None) => None, + Err(e) => Some((Err(e), stream)), + } + }), + ))) + } + + fn cardinality_effect(&self) -> CardinalityEffect { + CardinalityEffect::LowerEqual + } +} + +enum GroupJoinStreamState { + BuildRight, + ProbeAndAggregate, + Emit, + Done, +} + +struct RightData { + map: Arc, + batch: RecordBatch, + values: Vec, +} + +struct GroupJoinStream { + schema: SchemaRef, + on_left: Vec, + on_right: Vec, + + /// All right-side partition streams (collected exhaustively to build the hash table) + right_streams: Vec, + left_stream: Option, + + mode: AggregateMode, + group_by: PhysicalGroupBy, + aggregate_arguments: Vec>>, + accumulators: Vec>, + group_values: Box, + current_group_indices: Vec, + + batch_size: usize, + #[expect(dead_code)] + baseline_metrics: BaselineMetrics, + state: GroupJoinStreamState, + right_data: Option, + reservation: MemoryReservation, + random_state: RandomState, + hashes_buffer: Vec, +} + +impl GroupJoinStream { + async fn next_batch(&mut self) -> Result> { + loop { + match &self.state { + GroupJoinStreamState::BuildRight => { + self.build_right_side().await?; + self.state = GroupJoinStreamState::ProbeAndAggregate; + } + GroupJoinStreamState::ProbeAndAggregate => { + let left_stream = self.left_stream.as_mut().unwrap(); + match left_stream.next().await { + Some(Ok(batch)) => { + self.process_left_batch(&batch)?; + } + Some(Err(e)) => return Err(e), + None => { + self.state = GroupJoinStreamState::Emit; + } + } + } + GroupJoinStreamState::Emit => { + let result = self.emit_results()?; + self.state = GroupJoinStreamState::Done; + if result.num_rows() > 0 { + return Ok(Some(result)); + } + return Ok(None); + } + GroupJoinStreamState::Done => return Ok(None), + } + } + } + + async fn build_right_side(&mut self) -> Result<()> { + let right_streams = std::mem::take(&mut self.right_streams); + if right_streams.is_empty() { + return datafusion_common::internal_err!("Right streams already consumed"); + } + + let right_schema = right_streams[0].schema(); + let mut batches: Vec = Vec::new(); + let mut num_rows: usize = 0; + + // Drain all right-side partitions into a single batch collection. + for mut stream in right_streams { + while let Some(batch) = stream.next().await { + let batch = batch?; + let batch_mem = get_record_batch_memory_size(&batch); + self.reservation.try_grow(batch_mem)?; + num_rows += batch.num_rows(); + batches.push(batch); + } + } + + if num_rows == 0 { + self.right_data = Some(RightData { + map: Arc::new(Map::HashMap(Box::new(JoinHashMapU32::with_capacity(0)))), + batch: RecordBatch::new_empty(right_schema), + values: vec![], + }); + return Ok(()); + } + + let mut hashmap: Box = if num_rows > u32::MAX as usize { + Box::new(JoinHashMapU64::with_capacity(num_rows)) + } else { + Box::new(JoinHashMapU32::with_capacity(num_rows)) + }; + + let mut hash_buf = Vec::new(); + let mut offset = 0; + + for batch in batches.iter().rev() { + hash_buf.clear(); + hash_buf.resize(batch.num_rows(), 0); + update_hash( + &self.on_right, + batch, + &mut *hashmap, + offset, + &self.random_state, + &mut hash_buf, + 0, + true, + )?; + offset += batch.num_rows(); + } + + let batch = compute::concat_batches(&right_schema, batches.iter().rev())?; + let values = evaluate_expressions_to_arrays(&self.on_right, &batch)?; + + self.right_data = Some(RightData { + map: Arc::new(Map::HashMap(hashmap)), + batch, + values, + }); + + Ok(()) + } + + fn process_left_batch(&mut self, left_batch: &RecordBatch) -> Result<()> { + let right_data = self.right_data.as_ref().ok_or_else(|| { + datafusion_common::DataFusionError::Internal( + "Right data not built".to_string(), + ) + })?; + + if right_data.map.is_empty() { + return Ok(()); + } + + let left_key_values = evaluate_expressions_to_arrays(&self.on_left, left_batch)?; + + self.hashes_buffer.clear(); + self.hashes_buffer.resize(left_batch.num_rows(), 0); + create_hashes( + &left_key_values, + &self.random_state, + &mut self.hashes_buffer, + )?; + + let mut probe_indices_buf: Vec = Vec::new(); + let mut build_indices_buf: Vec = Vec::new(); + let mut offset: Option = Some((0, None)); + + // Pre-evaluate group-by and aggregate expressions once per batch + let group_by_values = evaluate_group_by(&self.group_by, left_batch)?; + + // Pre-evaluate aggregate arguments on the right batch + let right_arg_arrays: Vec> = self + .aggregate_arguments + .iter() + .map(|arg_exprs| { + arg_exprs + .iter() + .map(|expr| { + expr.evaluate(&right_data.batch) + .and_then(|v| v.into_array(right_data.batch.num_rows())) + }) + .collect::>>() + }) + .collect::>()?; + + while let Some(current_offset) = offset { + probe_indices_buf.clear(); + build_indices_buf.clear(); + + let (build_indices, probe_indices, next_offset) = + match right_data.map.as_ref() { + Map::HashMap(map) => lookup_join_hashmap( + map.as_ref(), + &right_data.values, + &left_key_values, + NullEquality::NullEqualsNothing, + &self.hashes_buffer, + self.batch_size, + current_offset, + &mut probe_indices_buf, + &mut build_indices_buf, + )?, + Map::ArrayMap(array_map) => { + let next = array_map.get_matched_indices_with_limit_offset( + &left_key_values, + self.batch_size, + current_offset, + &mut probe_indices_buf, + &mut build_indices_buf, + )?; + ( + UInt64Array::from(std::mem::take(&mut build_indices_buf)), + UInt32Array::from(std::mem::take(&mut probe_indices_buf)), + next, + ) + } + }; + + if build_indices.is_empty() { + offset = next_offset; + if offset.is_none() { + break; + } + continue; + } + + for group_values_arr in &group_by_values { + // Take group values at matched probe positions + let matched_group_values: Vec = group_values_arr + .iter() + .map(|arr| compute::take(arr.as_ref(), &probe_indices, None)) + .collect::>()?; + + self.group_values + .intern(&matched_group_values, &mut self.current_group_indices)?; + let total_num_groups = self.group_values.len(); + + // Feed matched right-side values to accumulators + for (acc_idx, acc) in self.accumulators.iter_mut().enumerate() { + let values: Vec = right_arg_arrays[acc_idx] + .iter() + .map(|arr| compute::take(arr.as_ref(), &build_indices, None)) + .collect::>()?; + + acc.update_batch( + &values, + &self.current_group_indices, + None, + total_num_groups, + )?; + } + } + + offset = next_offset; + } + + Ok(()) + } + + fn emit_results(&mut self) -> Result { + let group_columns = self.group_values.emit(EmitTo::All)?; + + let agg_columns: Vec = match self.mode.output_mode() { + AggregateOutputMode::Final => self + .accumulators + .iter_mut() + .map(|acc| acc.evaluate(EmitTo::All)) + .collect::>>()?, + AggregateOutputMode::Partial => self + .accumulators + .iter_mut() + .flat_map(|acc| match acc.state(EmitTo::All) { + Ok(states) => states.into_iter().map(Ok).collect::>(), + Err(e) => vec![Err(e)], + }) + .collect::>>()?, + }; + + let mut columns = group_columns; + columns.extend(agg_columns); + + if columns.is_empty() || columns[0].is_empty() { + return Ok(RecordBatch::new_empty(Arc::clone(&self.schema))); + } + + Ok(RecordBatch::try_new(Arc::clone(&self.schema), columns)?) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + + use arrow::array::{Int32Array, Int64Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_expr::expressions::Column; + + use crate::test::TestMemoryExec; + + fn build_task_context() -> Arc { + let runtime = RuntimeEnvBuilder::new().build_arc().unwrap(); + let config = SessionConfig::new().with_batch_size(4096); + Arc::new(TaskContext::new( + None, + "test_session".to_string(), + config, + Default::default(), + Default::default(), + Default::default(), + runtime, + )) + } + + #[tokio::test] + async fn test_group_join_basic() -> Result<()> { + // Left table: orders (order_id, customer_id) + let left_schema = Arc::new(Schema::new(vec![ + Field::new("order_id", DataType::Int32, false), + Field::new("customer_id", DataType::Int32, false), + ])); + let left_batch = RecordBatch::try_new( + Arc::clone(&left_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4])), + Arc::new(Int32Array::from(vec![10, 10, 20, 20])), + ], + )?; + + // Right table: line_items (order_id, amount) + let right_schema = Arc::new(Schema::new(vec![ + Field::new("order_id", DataType::Int32, false), + Field::new("amount", DataType::Int64, false), + ])); + let right_batch = RecordBatch::try_new( + Arc::clone(&right_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 1, 2, 3, 3, 3, 4])), + Arc::new(Int64Array::from(vec![100, 200, 300, 400, 500, 600, 700])), + ], + )?; + + let left_exec = Arc::new(TestMemoryExec::try_new( + &[vec![left_batch]], + Arc::clone(&left_schema), + None, + )?); + let right_exec = Arc::new(TestMemoryExec::try_new( + &[vec![right_batch]], + Arc::clone(&right_schema), + None, + )?); + + // GROUP BY customer_id, SUM(amount) + // Join on order_id + let on: JoinOn = vec![( + Arc::new(Column::new("order_id", 0)), + Arc::new(Column::new("order_id", 0)), + )]; + + let group_by_exprs = vec![( + Arc::new(Column::new("customer_id", 1)) as PhysicalExprRef, + "customer_id".to_string(), + )]; + + // SUM(amount) - amount is column 1 in the right schema + let sum_udf = datafusion_functions_aggregate::sum::sum_udaf(); + let sum_expr = Arc::new( + AggregateExprBuilder::new(sum_udf, vec![Arc::new(Column::new("amount", 1))]) + .schema(Arc::clone(&right_schema)) + .alias("sum_amount") + .build()?, + ); + + let output_schema = Arc::new(Schema::new(vec![ + Field::new("customer_id", DataType::Int32, false), + Field::new("sum_amount", DataType::Int64, true), + ])); + + let group_join = GroupJoinExec::try_new( + AggregateMode::Single, + left_exec, + right_exec, + on, + group_by_exprs, + vec![sum_expr], + output_schema, + )?; + + let context = build_task_context(); + let mut stream = group_join.execute(0, context)?; + + let mut results: Vec = Vec::new(); + while let Some(batch) = stream.next().await { + results.push(batch?); + } + + assert_eq!(results.len(), 1); + let result = &results[0]; + assert_eq!(result.num_rows(), 2); + + // Collect results into a map for order-independent comparison + let customer_ids = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let sums = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + + let mut result_map = std::collections::HashMap::new(); + for i in 0..result.num_rows() { + result_map.insert(customer_ids.value(i), sums.value(i)); + } + + // Customer 10 has orders 1, 2: + // order 1: amounts 100 + 200 = 300 + // order 2: amount 300 + // total = 600 + assert_eq!(result_map[&10], 600); + + // Customer 20 has orders 3, 4: + // order 3: amounts 400 + 500 + 600 = 1500 + // order 4: amount 700 + // total = 2200 + assert_eq!(result_map[&20], 2200); + + Ok(()) + } + + #[tokio::test] + async fn test_group_join_empty_right() -> Result<()> { + let left_schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, false), + Field::new("val", DataType::Int32, false), + ])); + let left_batch = RecordBatch::try_new( + Arc::clone(&left_schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(Int32Array::from(vec![10, 20])), + ], + )?; + + let right_schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, false), + Field::new("amount", DataType::Int64, false), + ])); + // Empty right side + let right_batch = RecordBatch::new_empty(Arc::clone(&right_schema)); + + let left_exec = Arc::new(TestMemoryExec::try_new( + &[vec![left_batch]], + Arc::clone(&left_schema), + None, + )?); + let right_exec = Arc::new(TestMemoryExec::try_new( + &[vec![right_batch]], + Arc::clone(&right_schema), + None, + )?); + + let on: JoinOn = vec![( + Arc::new(Column::new("key", 0)), + Arc::new(Column::new("key", 0)), + )]; + + let group_by_exprs = vec![( + Arc::new(Column::new("val", 1)) as PhysicalExprRef, + "val".to_string(), + )]; + + let sum_udf = datafusion_functions_aggregate::sum::sum_udaf(); + let sum_expr = Arc::new( + AggregateExprBuilder::new(sum_udf, vec![Arc::new(Column::new("amount", 1))]) + .schema(Arc::clone(&right_schema)) + .alias("sum_amount") + .build()?, + ); + + let output_schema = Arc::new(Schema::new(vec![ + Field::new("val", DataType::Int32, false), + Field::new("sum_amount", DataType::Int64, true), + ])); + + let group_join = GroupJoinExec::try_new( + AggregateMode::Single, + left_exec, + right_exec, + on, + group_by_exprs, + vec![sum_expr], + output_schema, + )?; + + let context = build_task_context(); + let mut stream = group_join.execute(0, context)?; + + let mut results: Vec = Vec::new(); + while let Some(batch) = stream.next().await { + results.push(batch?); + } + + // INNER join with empty right → no results + assert!(results.is_empty()); + + Ok(()) + } +} diff --git a/datafusion/physical-plan/src/joins/hash_join/mod.rs b/datafusion/physical-plan/src/joins/hash_join/mod.rs index b915802ea4015..d97451b3c0767 100644 --- a/datafusion/physical-plan/src/joins/hash_join/mod.rs +++ b/datafusion/physical-plan/src/joins/hash_join/mod.rs @@ -20,8 +20,8 @@ pub use exec::{HashJoinExec, HashJoinExecBuilder}; pub use partitioned_hash_eval::{HashExpr, HashTableLookupExpr, SeededRandomState}; -mod exec; +pub(crate) mod exec; mod inlist_builder; mod partitioned_hash_eval; mod shared_bounds; -mod stream; +pub(crate) mod stream; diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index ab630920184d3..5165ccb720fbb 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -283,7 +283,7 @@ impl RecordBatchStream for HashJoinStream { /// Probe indices: 3, 3, 4, 5 /// ``` #[expect(clippy::too_many_arguments)] -pub(super) fn lookup_join_hashmap( +pub(crate) fn lookup_join_hashmap( build_hashmap: &dyn JoinHashMapType, build_side_values: &[ArrayRef], probe_side_values: &[ArrayRef], diff --git a/datafusion/physical-plan/src/joins/mod.rs b/datafusion/physical-plan/src/joins/mod.rs index 2cdfa1e6ac020..0d99d60bcfcae 100644 --- a/datafusion/physical-plan/src/joins/mod.rs +++ b/datafusion/physical-plan/src/joins/mod.rs @@ -31,6 +31,7 @@ pub use sort_merge_join::SortMergeJoinExec; pub use symmetric_hash_join::SymmetricHashJoinExec; pub mod chain; mod cross_join; +pub mod group_join; mod hash_join; mod nested_loop_join; mod piecewise_merge_join; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 9916892058569..ef2a6941d2bb1 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -234,6 +234,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE @@ -314,6 +315,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE @@ -360,6 +362,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE @@ -605,6 +608,7 @@ physical_plan after OutputRequirements physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after GroupJoinOptimization SAME TEXT AS ABOVE physical_plan after FilterPushdown SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE diff --git a/datafusion/sqllogictest/test_files/group_join.slt b/datafusion/sqllogictest/test_files/group_join.slt new file mode 100644 index 0000000000000..7ae72b9284e72 --- /dev/null +++ b/datafusion/sqllogictest/test_files/group_join.slt @@ -0,0 +1,122 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# Tests for GroupJoin optimization +# (fusing GROUP BY + aggregation with INNER JOIN) + +# Setup test tables +statement ok +CREATE TABLE orders (order_id INT, customer_id INT) AS VALUES + (1, 10), + (2, 10), + (3, 20), + (4, 20); + +statement ok +CREATE TABLE line_items (order_id INT, amount BIGINT) AS VALUES + (1, 100), + (1, 200), + (2, 300), + (3, 400), + (3, 500), + (3, 600), + (4, 700); + +# Verify GroupJoinExec appears in the physical plan +query TT +EXPLAIN SELECT o.customer_id, SUM(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +logical_plan +01)Aggregate: groupBy=[[o.customer_id]], aggr=[[sum(li.amount)]] +02)--Projection: o.customer_id, li.amount +03)----Inner Join: o.order_id = li.order_id +04)------SubqueryAlias: o +05)--------TableScan: orders projection=[order_id, customer_id] +06)------SubqueryAlias: li +07)--------TableScan: line_items projection=[order_id, amount] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[customer_id@0 as customer_id], aggr=[sum(li.amount)] +02)--RepartitionExec: partitioning=Hash([customer_id@0], 4), input_partitions=1 +03)----GroupJoinExec: mode=Partial, on=[(order_id@0, order_id@0)], group_by=[customer_id@1 as customer_id], aggr=[sum(li.amount)] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------DataSourceExec: partitions=1, partition_sizes=[1] + +# Verify correctness: customer 10 = 100+200+300 = 600, customer 20 = 400+500+600+700 = 2200 +query II rowsort +SELECT o.customer_id, SUM(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +10 600 +20 2200 + +# GroupJoin with COUNT +query II rowsort +SELECT o.customer_id, COUNT(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +10 3 +20 4 + +# GroupJoin with multiple aggregates +query IIRI rowsort +SELECT o.customer_id, SUM(li.amount), AVG(li.amount), COUNT(*) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.customer_id; +---- +10 600 200 3 +20 2200 550 4 + +# GroupJoin with GROUP BY = join key (simplest case) +query II rowsort +SELECT o.order_id, SUM(li.amount) +FROM orders o +JOIN line_items li ON o.order_id = li.order_id +GROUP BY o.order_id; +---- +1 300 +2 300 +3 1500 +4 700 + +# No matching rows scenario +statement ok +CREATE TABLE empty_items (order_id INT, amount BIGINT) AS VALUES (99, 0); + +query II rowsort +SELECT o.customer_id, SUM(ei.amount) +FROM orders o +JOIN empty_items ei ON o.order_id = ei.order_id +GROUP BY o.customer_id; +---- + +# Cleanup +statement ok +DROP TABLE orders; + +statement ok +DROP TABLE line_items; + +statement ok +DROP TABLE empty_items; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 136a68573562a..d1e729cfd2b17 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1340,11 +1340,11 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[] 02)--RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +03)----GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as t1_id], aggr=[] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)--------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1395,11 +1395,11 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +04)------GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1422,11 +1422,11 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +07)------------GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as alias1], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------------DataSourceExec: partitions=1, partition_sizes=[1] +10)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index b152fde02f060..d5f82737baecd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -64,10 +64,7 @@ physical_plan 03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 05)--------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] -08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -09)----------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false -11)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false +06)----------GroupJoinExec: mode=Partial, on=[(l_orderkey@0, o_orderkey@0)], group_by=[l_shipmode@1 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +07)------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] +08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false +09)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false From ad8fd7499a39b3648da09b7cff8eba4f3fece6ff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 17 Mar 2026 22:57:15 +0100 Subject: [PATCH 3/8] Extra --- .../physical-optimizer/src/group_join.rs | 92 +++- .../physical-plan/src/joins/group_join.rs | 514 +++++++++++++----- 2 files changed, 434 insertions(+), 172 deletions(-) diff --git a/datafusion/physical-optimizer/src/group_join.rs b/datafusion/physical-optimizer/src/group_join.rs index 056a22c81652a..d38a14a18bbfe 100644 --- a/datafusion/physical-optimizer/src/group_join.rs +++ b/datafusion/physical-optimizer/src/group_join.rs @@ -32,7 +32,7 @@ use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::joins::HashJoinExec; -use datafusion_physical_plan::joins::group_join::GroupJoinExec; +use datafusion_physical_plan::joins::group_join::{GroupBySide, GroupJoinExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::{ExecutionPlan, PhysicalExpr}; @@ -219,46 +219,81 @@ fn try_create_group_join( Some(key_subst.get(&resolved).copied().unwrap_or(resolved)) }; - // Group-by expressions must all resolve to left-side join columns (after substitution) + // Classify each group-by column as left-side or right-side. + // Left-side: resolves (with key substitution) to left schema. + // Right-side: resolves to right schema (any right column, not just join keys). + let mut group_by_order: Vec = Vec::new(); for (expr, _alias) in group_by.expr() { - if !expr_columns_in_side(expr.as_ref(), &resolve_gby, 0, left_field_count) { + if expr_columns_in_side(expr.as_ref(), &resolve_gby, 0, left_field_count) { + group_by_order.push(GroupBySide::Left); + } else if expr_columns_in_side( + expr.as_ref(), + &*resolve_index, + left_field_count, + usize::MAX, + ) { + group_by_order.push(GroupBySide::Right); + } else { + // Expression references neither side cleanly (e.g. spans both) return Ok(None); } } - // Aggregate arguments must all resolve to right-side join columns + // Classify each aggregate's argument expressions: all must resolve to a single side. + let mut aggr_arg_sides: Vec = Vec::new(); for agg_expr in agg_exec.aggr_expr() { - for arg in agg_expr.expressions() { - if !expr_columns_in_side( - arg.as_ref(), - &*resolve_index, - left_field_count, - usize::MAX, - ) { - return Ok(None); - } + let args = agg_expr.expressions(); + if args.is_empty() { + // COUNT(*) style — treat as right side (no actual args to evaluate) + aggr_arg_sides.push(GroupBySide::Right); + continue; + } + let all_right = args.iter().all(|arg| { + expr_columns_in_side(arg.as_ref(), &*resolve_index, left_field_count, usize::MAX) + }); + let all_left = args.iter().all(|arg| { + expr_columns_in_side(arg.as_ref(), &*resolve_index, 0, left_field_count) + }); + if all_right { + aggr_arg_sides.push(GroupBySide::Right); + } else if all_left { + aggr_arg_sides.push(GroupBySide::Left); + } else { + // Args span both sides — cannot fuse + return Ok(None); } } - // Remap group-by expressions to reference the left input schema directly (with substitution) - let remapped_group_by: Vec<_> = group_by - .expr() - .iter() - .map(|(expr, alias)| { - let remapped = remap_columns(expr, &resolve_gby, 0); - (remapped, alias.clone()) - }) - .collect(); + // Remap group-by into left and right lists, preserving the group_by_order mapping + let mut left_gby: Vec<(Arc, String)> = Vec::new(); + let mut right_gby: Vec<(Arc, String)> = Vec::new(); + for ((expr, alias), &side) in group_by.expr().iter().zip(group_by_order.iter()) { + match side { + GroupBySide::Left => { + left_gby.push((remap_columns(expr, &resolve_gby, 0), alias.clone())); + } + GroupBySide::Right => { + right_gby + .push((remap_columns(expr, &*resolve_index, left_field_count), alias.clone())); + } + } + } - // Remap aggregate arguments to reference the right input schema directly + // Remap aggregate arguments to reference the appropriate input schema let remapped_aggr_exprs: Option> = agg_exec .aggr_expr() .iter() - .map(|expr| { + .zip(aggr_arg_sides.iter()) + .map(|(expr, &side)| { + let offset = if side == GroupBySide::Right { + left_field_count + } else { + 0 + }; let remapped_args: Vec> = expr .expressions() .iter() - .map(|a| remap_columns(a, &*resolve_index, left_field_count)) + .map(|a| remap_columns(a, &*resolve_index, offset)) .collect(); expr.with_new_expressions(remapped_args, vec![]) }) @@ -270,13 +305,16 @@ fn try_create_group_join( let remapped_aggr_exprs: Vec<_> = remapped_aggr_exprs.into_iter().map(Arc::new).collect(); - GroupJoinExec::try_new( + GroupJoinExec::try_new_extended( *agg_exec.mode(), Arc::clone(hash_join.left()), Arc::clone(hash_join.right()), hash_join.on().to_vec(), - remapped_group_by, + left_gby, + right_gby, + group_by_order, remapped_aggr_exprs, + aggr_arg_sides, Arc::clone(&agg_exec.schema()), ) .map(Some) diff --git a/datafusion/physical-plan/src/joins/group_join.rs b/datafusion/physical-plan/src/joins/group_join.rs index 15d84c86b6a8d..58804a6f63707 100644 --- a/datafusion/physical-plan/src/joins/group_join.rs +++ b/datafusion/physical-plan/src/joins/group_join.rs @@ -32,6 +32,7 @@ use std::sync::Arc; use arrow::array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; use arrow::compute; +use arrow::compute::concat_batches; use arrow::datatypes::SchemaRef; use datafusion_common::hash_utils::{RandomState, create_hashes}; use datafusion_common::tree_node::TreeNodeRecursion; @@ -43,18 +44,17 @@ use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::PhysicalExprRef; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::utils::evaluate_expressions_to_arrays; +use futures::future::poll_fn; use futures::StreamExt; use crate::aggregates::group_values::{GroupValues, new_group_values}; use crate::aggregates::order::GroupOrdering; use crate::aggregates::row_hash::create_group_accumulator; -use crate::aggregates::{ - AggregateMode, AggregateOutputMode, PhysicalGroupBy, evaluate_group_by, -}; +use crate::aggregates::{AggregateMode, AggregateOutputMode}; use crate::execution_plan::{CardinalityEffect, EmissionType, PlanProperties}; use crate::joins::hash_join::stream::lookup_join_hashmap; use crate::joins::join_hash_map::{JoinHashMapU32, JoinHashMapU64}; -use crate::joins::utils::{JoinHashMapType, update_hash}; +use crate::joins::utils::{JoinHashMapType, OnceAsync, OnceFut, update_hash}; use crate::joins::{JoinOn, Map, MapOffset}; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; use crate::stream::RecordBatchStreamAdapter; @@ -63,37 +63,76 @@ use crate::{ SendableRecordBatchStream, }; +/// Indicates which input side a group-by column or aggregate argument comes from. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum GroupBySide { + Left, + Right, +} + +/// Built right-side data: hash map, concatenated batch, join-key values, and group-by arrays. +/// Computed once and shared (via `Arc`) across all left-side partitions. +struct RightData { + map: Arc, + batch: RecordBatch, + /// Evaluated join key columns (for hash lookups) + values: Vec, + /// Evaluated right-side group-by columns (for group key construction) + group_by_arrays: Vec, +} + +impl fmt::Debug for RightData { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("RightData") + .field("num_rows", &self.batch.num_rows()) + .finish() + } +} + /// GroupJoinExec fuses a hash join with a subsequent aggregation. /// /// Instead of materializing the full join result and then grouping, /// it builds a hash table on the right (build) side and, for each /// left (probe) row, directly aggregates matching right-side values. /// +/// Group-by columns may reference either the left (probe) or right (build) input. +/// Aggregate argument expressions are evaluated on the side specified by `aggr_arg_sides`. +/// /// This is beneficial when the join has high fan-out (many right rows /// per left row) but the final result is much smaller after aggregation. #[derive(Debug)] pub struct GroupJoinExec { - /// Aggregation mode (Single or Partial) + /// Aggregation mode (Partial, Single, etc.) mode: AggregateMode, - /// Left input (probe side) — group-by columns come from here + /// Left input (probe side) left: Arc, - /// Right input (build side) — aggregated columns come from here + /// Right input (build side) right: Arc, /// Equi-join keys: (left_expr, right_expr) on: JoinOn, - /// Group-by expressions evaluated on the left (probe) input - group_by: PhysicalGroupBy, - /// Aggregate function expressions with arguments referencing the right (build) input + /// Group-by expressions referencing the LEFT (probe) input schema + left_group_by: Vec<(PhysicalExprRef, String)>, + /// Group-by expressions referencing the RIGHT (build) input schema + right_group_by: Vec<(PhysicalExprRef, String)>, + /// Interleave order for output group-by columns: Left(left_group_by[i]) or Right(right_group_by[j]) + group_by_order: Vec, + /// Aggregate function expressions aggr_expr: Vec>, + /// For each aggr_expr, which input side provides the argument values + aggr_arg_sides: Vec, /// Output schema (matches the original AggregateExec's output) schema: SchemaRef, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cached plan properties cache: Arc, + /// Shared right-side hash table — built once, reused by all left partitions + right_fut: Arc>>, } impl GroupJoinExec { + /// Create a GroupJoinExec where all group-by columns come from the left (probe) side + /// and all aggregate arguments come from the right (build) side. pub fn try_new( mode: AggregateMode, left: Arc, @@ -103,25 +142,84 @@ impl GroupJoinExec { aggr_expr: Vec>, schema: SchemaRef, ) -> Result { - let group_by = PhysicalGroupBy::new_single(group_by_exprs); + let n = group_by_exprs.len(); + let a = aggr_expr.len(); + Self::try_new_extended( + mode, + left, + right, + on, + group_by_exprs, + vec![], + vec![GroupBySide::Left; n], + aggr_expr, + vec![GroupBySide::Right; a], + schema, + ) + } + + /// Create a GroupJoinExec with full control over which side each group-by column + /// and aggregate argument comes from. + /// + /// - `left_group_by`: expressions referencing the left (probe) input schema + /// - `right_group_by`: expressions referencing the right (build) input schema + /// - `group_by_order`: for each output group-by column, `Left` or `Right` + /// (consumed left-to-right within each side's list) + /// - `aggr_arg_sides`: for each aggregate expression, which input side provides args + pub fn try_new_extended( + mode: AggregateMode, + left: Arc, + right: Arc, + on: JoinOn, + left_group_by: Vec<(PhysicalExprRef, String)>, + right_group_by: Vec<(PhysicalExprRef, String)>, + group_by_order: Vec, + aggr_expr: Vec>, + aggr_arg_sides: Vec, + schema: SchemaRef, + ) -> Result { let cache = Arc::new(Self::compute_properties( left.properties(), Arc::clone(&schema), )); - Ok(Self { mode, left, right, on, - group_by, + left_group_by, + right_group_by, + group_by_order, aggr_expr, + aggr_arg_sides, schema, metrics: ExecutionPlanMetricsSet::new(), cache, + right_fut: Arc::new(OnceAsync::default()), }) } + /// Reconstruct the full group-by list in output order, as (expr, alias, side) triples. + fn ordered_group_by(&self) -> Vec<(&PhysicalExprRef, &str, GroupBySide)> { + let mut left_i = 0; + let mut right_i = 0; + self.group_by_order + .iter() + .map(|&side| match side { + GroupBySide::Left => { + let (e, a) = &self.left_group_by[left_i]; + left_i += 1; + (e, a.as_str(), GroupBySide::Left) + } + GroupBySide::Right => { + let (e, a) = &self.right_group_by[right_i]; + right_i += 1; + (e, a.as_str(), GroupBySide::Right) + } + }) + .collect() + } + fn compute_properties( left_props: &PlanProperties, schema: SchemaRef, @@ -148,10 +246,9 @@ impl DisplayAs for GroupJoinExec { .collect::>() .join(", "); let group_by: Vec<_> = self - .group_by - .expr() + .ordered_group_by() .iter() - .map(|(e, alias)| format!("{e} as {alias}")) + .map(|(e, alias, _)| format!("{e} as {alias}")) .collect(); let aggrs: Vec<_> = self .aggr_expr @@ -176,10 +273,9 @@ impl DisplayAs for GroupJoinExec { .join(", "); writeln!(f, "on={on}")?; let group_by: Vec<_> = self - .group_by - .expr() + .ordered_group_by() .iter() - .map(|(e, alias)| format!("{e} as {alias}")) + .map(|(e, alias, _)| format!("{e} as {alias}")) .collect(); writeln!(f, "group_by=[{}]", group_by.join(", "))?; let aggrs: Vec<_> = self @@ -224,7 +320,10 @@ impl ExecutionPlan for GroupJoinExec { tnr = tnr.visit_sibling(|| f(left.as_ref()))?; tnr = tnr.visit_sibling(|| f(right.as_ref()))?; } - for (expr, _) in self.group_by.expr() { + for (expr, _) in &self.left_group_by { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } + for (expr, _) in &self.right_group_by { tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; } for agg in &self.aggr_expr { @@ -239,13 +338,17 @@ impl ExecutionPlan for GroupJoinExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(GroupJoinExec::try_new( + // Reset right_fut so the new plan re-collects the right side from scratch + Ok(Arc::new(GroupJoinExec::try_new_extended( self.mode, Arc::clone(&children[0]), Arc::clone(&children[1]), self.on.clone(), - self.group_by.expr().to_vec(), + self.left_group_by.clone(), + self.right_group_by.clone(), + self.group_by_order.clone(), self.aggr_expr.clone(), + self.aggr_arg_sides.clone(), Arc::clone(&self.schema), )?)) } @@ -255,20 +358,35 @@ impl ExecutionPlan for GroupJoinExec { partition: usize, context: Arc, ) -> Result { - let on_right: Vec = - self.on.iter().map(|on| Arc::clone(&on.1)).collect(); let on_left: Vec = self.on.iter().map(|on| Arc::clone(&on.0)).collect(); - // Collect ALL right-side partitions so any left-side partitioning is safe, - // mirroring HashJoinExec(CollectLeft) which collects the full build side. - let right_part_count = self.right.output_partitioning().partition_count(); - let right_streams: Vec = (0..right_part_count) - .map(|p| self.right.execute(p, Arc::clone(&context))) - .collect::>()?; - let left_stream = self.left.execute(partition, Arc::clone(&context))?; + // Build right-side hash table exactly once, shared across all left partitions. + // This mirrors HashJoinExec(CollectLeft) — the build side is computed once. + let right_fut: OnceFut> = { + let right = Arc::clone(&self.right); + let on_right: Vec = + self.on.iter().map(|on| Arc::clone(&on.1)).collect(); + let right_group_by_exprs: Vec = self + .right_group_by + .iter() + .map(|(e, _)| Arc::clone(e)) + .collect(); + let random_state = RandomState::with_seed(0); + let ctx = Arc::clone(&context); + self.right_fut.try_once(|| { + Ok(collect_right_input( + right, + on_right, + right_group_by_exprs, + random_state, + ctx, + )) + })? + }; + let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); let accumulators: Vec> = self @@ -277,29 +395,44 @@ impl ExecutionPlan for GroupJoinExec { .map(create_group_accumulator) .collect::>()?; - let aggregate_arguments: Vec>> = - self.aggr_expr.iter().map(|agg| agg.expressions()).collect(); + // For each aggregate, collect its argument expressions paired with side info + let agg_args: Vec<(Vec>, GroupBySide)> = self + .aggr_expr + .iter() + .zip(self.aggr_arg_sides.iter()) + .map(|(agg, &side)| (agg.expressions(), side)) + .collect(); + + let left_group_by_exprs: Vec = + self.left_group_by.iter().map(|(e, _)| Arc::clone(e)).collect(); + let group_by_order = self.group_by_order.clone(); - let group_by = self.group_by.clone(); let mode = self.mode; let schema = Arc::clone(&self.schema); let batch_size = context.session_config().batch_size(); - let group_schema = group_by.group_schema(&self.left.schema())?; + // Build group schema from the first N fields of the output schema + // (AggregateExec always places group-by fields first) + let num_group_cols = self.left_group_by.len() + self.right_group_by.len(); + let group_schema = Arc::new(arrow::datatypes::Schema::new( + self.schema.fields()[..num_group_cols].to_vec(), + )); let group_values = new_group_values(group_schema, &GroupOrdering::None)?; let reservation = MemoryConsumer::new("GroupJoinExec").register(context.memory_pool()); + let random_state = RandomState::with_seed(0); + let stream = GroupJoinStream { schema: Arc::clone(&schema), on_left, - on_right, - right_streams, + right_fut, left_stream: Some(left_stream), mode, - group_by, - aggregate_arguments, + left_group_by_exprs, + group_by_order, + agg_args, accumulators, group_values, current_group_indices: Vec::new(), @@ -308,8 +441,8 @@ impl ExecutionPlan for GroupJoinExec { state: GroupJoinStreamState::BuildRight, right_data: None, reservation, - random_state: RandomState::with_seed(0), hashes_buffer: Vec::new(), + random_state, }; Ok(Box::pin(RecordBatchStreamAdapter::new( @@ -336,24 +469,21 @@ enum GroupJoinStreamState { Done, } -struct RightData { - map: Arc, - batch: RecordBatch, - values: Vec, -} - struct GroupJoinStream { schema: SchemaRef, on_left: Vec, - on_right: Vec, - /// All right-side partition streams (collected exhaustively to build the hash table) - right_streams: Vec, + /// Future that resolves to the shared right-side hash table (built once) + right_fut: OnceFut>, left_stream: Option, mode: AggregateMode, - group_by: PhysicalGroupBy, - aggregate_arguments: Vec>>, + /// Group-by expressions evaluated against the LEFT (probe) input + left_group_by_exprs: Vec, + /// Interleave order for group-by output columns + group_by_order: Vec, + /// Per-aggregate: (arg expressions, which input side to evaluate them on) + agg_args: Vec<(Vec>, GroupBySide)>, accumulators: Vec>, group_values: Box, current_group_indices: Vec, @@ -362,10 +492,11 @@ struct GroupJoinStream { #[expect(dead_code)] baseline_metrics: BaselineMetrics, state: GroupJoinStreamState, - right_data: Option, + /// Resolved right-side data (set after BuildRight state completes) + right_data: Option>, reservation: MemoryReservation, - random_state: RandomState, hashes_buffer: Vec, + random_state: RandomState, } impl GroupJoinStream { @@ -402,69 +533,17 @@ impl GroupJoinStream { } async fn build_right_side(&mut self) -> Result<()> { - let right_streams = std::mem::take(&mut self.right_streams); - if right_streams.is_empty() { - return datafusion_common::internal_err!("Right streams already consumed"); - } - - let right_schema = right_streams[0].schema(); - let mut batches: Vec = Vec::new(); - let mut num_rows: usize = 0; - - // Drain all right-side partitions into a single batch collection. - for mut stream in right_streams { - while let Some(batch) = stream.next().await { - let batch = batch?; - let batch_mem = get_record_batch_memory_size(&batch); - self.reservation.try_grow(batch_mem)?; - num_rows += batch.num_rows(); - batches.push(batch); - } - } - - if num_rows == 0 { - self.right_data = Some(RightData { - map: Arc::new(Map::HashMap(Box::new(JoinHashMapU32::with_capacity(0)))), - batch: RecordBatch::new_empty(right_schema), - values: vec![], - }); - return Ok(()); - } - - let mut hashmap: Box = if num_rows > u32::MAX as usize { - Box::new(JoinHashMapU64::with_capacity(num_rows)) - } else { - Box::new(JoinHashMapU32::with_capacity(num_rows)) - }; - - let mut hash_buf = Vec::new(); - let mut offset = 0; - - for batch in batches.iter().rev() { - hash_buf.clear(); - hash_buf.resize(batch.num_rows(), 0); - update_hash( - &self.on_right, - batch, - &mut *hashmap, - offset, - &self.random_state, - &mut hash_buf, - 0, - true, - )?; - offset += batch.num_rows(); - } - - let batch = compute::concat_batches(&right_schema, batches.iter().rev())?; - let values = evaluate_expressions_to_arrays(&self.on_right, &batch)?; - - self.right_data = Some(RightData { - map: Arc::new(Map::HashMap(hashmap)), - batch, - values, - }); - + // Await the shared right-side future (built once across all left partitions). + // OnceFut uses a poll-based API, so we bridge it via poll_fn. + let right_data: Arc = poll_fn(|cx| { + self.right_fut + .get(cx) + .map(|r| r.map(|arc| Arc::clone(arc))) + }) + .await?; + let batch_mem = get_record_batch_memory_size(&right_data.batch); + self.reservation.try_grow(batch_mem)?; + self.right_data = Some(right_data); Ok(()) } @@ -493,15 +572,23 @@ impl GroupJoinStream { let mut build_indices_buf: Vec = Vec::new(); let mut offset: Option = Some((0, None)); - // Pre-evaluate group-by and aggregate expressions once per batch - let group_by_values = evaluate_group_by(&self.group_by, left_batch)?; + // Pre-evaluate left-side group-by columns against the current left batch + let left_gby_arrays: Vec = self + .left_group_by_exprs + .iter() + .map(|expr| { + expr.evaluate(left_batch) + .and_then(|v| v.into_array(left_batch.num_rows())) + }) + .collect::>()?; - // Pre-evaluate aggregate arguments on the right batch - let right_arg_arrays: Vec> = self - .aggregate_arguments + // Pre-evaluate right-side aggregate arguments (right side is already built) + let right_agg_arrays: Vec> = self + .agg_args .iter() - .map(|arg_exprs| { - arg_exprs + .filter(|(_, side)| *side == GroupBySide::Right) + .map(|(exprs, _)| { + exprs .iter() .map(|expr| { expr.evaluate(&right_data.batch) @@ -552,31 +639,79 @@ impl GroupJoinStream { continue; } - for group_values_arr in &group_by_values { - // Take group values at matched probe positions - let matched_group_values: Vec = group_values_arr - .iter() - .map(|arr| compute::take(arr.as_ref(), &probe_indices, None)) - .collect::>()?; + // Build the ordered group-by array list by interleaving left and right + let left_taken: Vec = left_gby_arrays + .iter() + .map(|arr| compute::take(arr.as_ref(), &probe_indices, None)) + .collect::>()?; + let right_taken: Vec = right_data + .group_by_arrays + .iter() + .map(|arr| compute::take(arr.as_ref(), &build_indices, None)) + .collect::>()?; + + let mut left_i = 0; + let mut right_i = 0; + let matched_group_values: Vec = self + .group_by_order + .iter() + .map(|side| match side { + GroupBySide::Left => { + let v = Arc::clone(&left_taken[left_i]); + left_i += 1; + v + } + GroupBySide::Right => { + let v = Arc::clone(&right_taken[right_i]); + right_i += 1; + v + } + }) + .collect(); - self.group_values - .intern(&matched_group_values, &mut self.current_group_indices)?; - let total_num_groups = self.group_values.len(); + self.group_values + .intern(&matched_group_values, &mut self.current_group_indices)?; + let total_num_groups = self.group_values.len(); - // Feed matched right-side values to accumulators - for (acc_idx, acc) in self.accumulators.iter_mut().enumerate() { - let values: Vec = right_arg_arrays[acc_idx] + // Feed matched values to accumulators (left args taken at probe_indices, right at build_indices) + let mut right_agg_idx = 0; + for (acc_idx, (acc, (arg_exprs, side))) in self + .accumulators + .iter_mut() + .zip(self.agg_args.iter()) + .enumerate() + { + let _ = acc_idx; + let values: Vec = match side { + GroupBySide::Left => arg_exprs .iter() - .map(|arr| compute::take(arr.as_ref(), &build_indices, None)) - .collect::>()?; - - acc.update_batch( - &values, - &self.current_group_indices, - None, - total_num_groups, - )?; - } + .map(|expr| { + expr.evaluate(left_batch) + .and_then(|v| v.into_array(left_batch.num_rows())) + .and_then(|arr| { + compute::take(arr.as_ref(), &probe_indices, None) + .map_err(Into::into) + }) + }) + .collect::>()?, + GroupBySide::Right => { + let arrays = &right_agg_arrays[right_agg_idx]; + right_agg_idx += 1; + arrays + .iter() + .map(|arr| { + compute::take(arr.as_ref(), &build_indices, None) + .map_err(Into::into) + }) + .collect::>()? + } + }; + acc.update_batch( + &values, + &self.current_group_indices, + None, + total_num_groups, + )?; } offset = next_offset; @@ -586,6 +721,9 @@ impl GroupJoinStream { } fn emit_results(&mut self) -> Result { + if self.group_values.len() == 0 { + return Ok(RecordBatch::new_empty(Arc::clone(&self.schema))); + } let group_columns = self.group_values.emit(EmitTo::All)?; let agg_columns: Vec = match self.mode.output_mode() { @@ -615,6 +753,92 @@ impl GroupJoinStream { } } +/// Collects all right-side (build) partitions into a single `RightData`. +/// +/// Mirrors the pattern used by `HashJoinExec(CollectLeft)`: +/// - Executes all right partitions and concatenates their batches. +/// - Builds a hash map keyed on the equi-join columns. +/// - Evaluates right-side group-by expressions once on the combined batch. +/// +/// The resulting `Arc` is shared across all left (probe) partitions +/// via `OnceAsync`, so the right side is collected exactly once. +async fn collect_right_input( + right: Arc, + on_right: Vec, + right_group_by_exprs: Vec, + random_state: RandomState, + context: Arc, +) -> Result> { + let num_partitions = right.output_partitioning().partition_count(); + let schema = right.schema(); + + // Execute all right partitions and collect all batches. + let mut all_batches: Vec = Vec::new(); + for p in 0..num_partitions { + let mut stream = right.execute(p, Arc::clone(&context))?; + while let Some(batch) = stream.next().await { + let batch = batch?; + if batch.num_rows() > 0 { + all_batches.push(batch); + } + } + } + + if all_batches.is_empty() { + // Empty right side: return an empty RightData with an empty map. + let empty_batch = RecordBatch::new_empty(schema); + let hashmap = Box::new(JoinHashMapU32::with_capacity(0)); + return Ok(Arc::new(RightData { + map: Arc::new(Map::HashMap(hashmap)), + batch: empty_batch, + values: vec![], + group_by_arrays: vec![], + })); + } + + let num_rows: usize = all_batches.iter().map(|b| b.num_rows()).sum(); + + // Build the hash map (process batches in reverse order, as HashJoinExec does). + let mut hashmap: Box = if num_rows > u32::MAX as usize { + Box::new(JoinHashMapU64::with_capacity(num_rows)) + } else { + Box::new(JoinHashMapU32::with_capacity(num_rows)) + }; + let mut hashes_buffer: Vec = Vec::new(); + let mut offset = 0usize; + for batch in all_batches.iter().rev() { + hashes_buffer.clear(); + hashes_buffer.resize(batch.num_rows(), 0); + update_hash( + &on_right, + batch, + &mut *hashmap, + offset, + &random_state, + &mut hashes_buffer, + 0, + true, + )?; + offset += batch.num_rows(); + } + + // Concatenate all right batches into one (for index-based lookups). + let batch = concat_batches(&schema, all_batches.iter().rev())?; + + // Evaluate join key columns on the concatenated batch. + let values = evaluate_expressions_to_arrays(&on_right, &batch)?; + + // Evaluate right-side group-by columns on the concatenated batch. + let group_by_arrays = evaluate_expressions_to_arrays(&right_group_by_exprs, &batch)?; + + Ok(Arc::new(RightData { + map: Arc::new(Map::HashMap(hashmap)), + batch, + values, + group_by_arrays, + })) +} + #[cfg(test)] mod tests { use super::*; From de1619a253ed55c2bc57ac6791a66fcfb77f9a0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 17 Mar 2026 23:09:31 +0100 Subject: [PATCH 4/8] Extra --- .../repartition_subset_satisfaction.slt | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index e2c9fa4237939..d2d58a771a99d 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -374,16 +374,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -09)----------------SortExec: expr=[f_dkey@0 ASC NULLS LAST, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] -10)------------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 -11)--------------------AggregateExec: mode=Partial, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -12)----------------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] -13)------------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] -14)--------------------------CoalescePartitionsExec -15)----------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -16)------------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -17)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)] +09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 +10)------------------GroupJoinExec: mode=Partial, on=[(d_dkey@1, f_dkey@2)], group_by=[f_dkey@1 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@0) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@0 as env], aggr=[max(j.value)] +11)--------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] +12)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +13)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet # Verify results without subset satisfaction query TPR rowsort @@ -473,13 +469,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] -08)--------------AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -09)----------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] -10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] -11)--------------------CoalescePartitionsExec -12)----------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -13)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -14)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)] +09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 +10)------------------GroupJoinExec: mode=Partial, on=[(d_dkey@1, f_dkey@2)], group_by=[f_dkey@1 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@0) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@0 as env], aggr=[max(j.value)] +11)--------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] +12)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +13)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet # Verify results match with subset satisfaction query TPR rowsort From 1bfb5de8252cbad3548cdfa4a0c5c34f8ce388fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 18 Mar 2026 06:16:15 +0100 Subject: [PATCH 5/8] fix: GroupJoin build/probe side swap and pre-compute build-side agg arrays Two performance fixes for GroupJoinExec: 1. Build hash table on correct side: In CollectLeft mode, HashJoinExec builds on the LEFT (small) side, but GroupJoinExec builds on its RIGHT child. The optimizer now swaps inputs so GroupJoinExec.right = the original build side (small), fixing cases like TPC-H Q3 where a 200x larger hash table was being built on the wrong side. 2. Pre-compute build-side aggregate arrays: Previously, build-side aggregate argument expressions were re-evaluated on the entire build batch for every probe batch. Now they are evaluated once during collect_right_input() and stored in RightData. Also restricts GroupJoin to only match CollectLeft mode, since Partitioned mode has per-partition semantics incompatible with GroupJoinExec's collect-all-right behavior. Co-Authored-By: Claude Opus 4.6 (1M context) --- .../physical-optimizer/src/group_join.rs | 123 +++++++++++++----- .../physical-plan/src/joins/group_join.rs | 67 ++++++---- datafusion/sqllogictest/test_files/joins.slt | 6 +- .../test_files/tpch/plans/q10.slt.part | 30 ++--- .../test_files/tpch/plans/q11.slt.part | 51 ++++---- .../test_files/tpch/plans/q12.slt.part | 8 +- .../test_files/tpch/plans/q2.slt.part | 29 ++--- .../test_files/tpch/plans/q5.slt.part | 47 ++++--- 8 files changed, 211 insertions(+), 150 deletions(-) diff --git a/datafusion/physical-optimizer/src/group_join.rs b/datafusion/physical-optimizer/src/group_join.rs index d38a14a18bbfe..f31b06fc78a5d 100644 --- a/datafusion/physical-optimizer/src/group_join.rs +++ b/datafusion/physical-optimizer/src/group_join.rs @@ -33,6 +33,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::joins::HashJoinExec; use datafusion_physical_plan::joins::group_join::{GroupBySide, GroupJoinExec}; +use datafusion_physical_plan::joins::{JoinOn, PartitionMode}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::{ExecutionPlan, PhysicalExpr}; @@ -166,6 +167,12 @@ fn try_create_group_join( return Ok(None); } + // Only support CollectLeft mode — Partitioned mode has per-partition + // semantics that GroupJoinExec (which collects the entire build side) doesn't match. + if *hash_join.partition_mode() != PartitionMode::CollectLeft { + return Ok(None); + } + let group_by = agg_exec.group_expr(); if group_by.has_grouping_set() { return Ok(None); @@ -195,12 +202,30 @@ fn try_create_group_join( // The join output schema (before any projection) is [left_cols..., right_cols...] let left_field_count = hash_join.left().schema().fields().len(); + let right_field_count = hash_join.right().schema().fields().len(); let resolve_index = build_resolver(hash_join, proj_exec); - // Build a substitution map: right-side join-key (raw join schema index) → left-side key index. - // When a GROUP BY column references a right join key `right_j`, and there is an ON condition - // `left_i = right_j`, we can substitute it with `left_i` (they are equal by the join predicate). + // In CollectLeft mode: + // - HashJoinExec: LEFT is the build side (collected), RIGHT is probe (streamed). + // - GroupJoinExec: RIGHT is build, LEFT is probe. + // So we swap: GroupJoinExec.left = HashJoinExec.right, + // GroupJoinExec.right = HashJoinExec.left. + // + // In the join output schema: + // - Build columns (HashJoin.left): [0, left_field_count) + // - Probe columns (HashJoin.right): [left_field_count, left_field_count + right_field_count) + // + // GroupBySide::Left corresponds to the probe side (GroupJoinExec.left = HashJoin.right). + // GroupBySide::Right corresponds to the build side (GroupJoinExec.right = HashJoin.left). + let probe_range_start = left_field_count; + let probe_range_end = left_field_count + right_field_count; + let build_range_start = 0usize; + let build_range_end = left_field_count; + + // Build a substitution map: build-side join keys → probe-side join keys. + // When a GROUP BY column references a build-side join key, we can substitute + // it with the equivalent probe-side key (equal by the equi-join predicate). // This is Section 3.2 of Moerkotte & Neumann VLDB 2011. let key_subst: HashMap = hash_join .on() @@ -208,33 +233,38 @@ fn try_create_group_join( .filter_map(|(l_expr, r_expr)| { let l_col = l_expr.as_any().downcast_ref::()?; let r_col = r_expr.as_any().downcast_ref::()?; - Some((left_field_count + r_col.index(), l_col.index())) + // Build side = HashJoin.left, Probe side = HashJoin.right + Some((l_col.index(), left_field_count + r_col.index())) }) .collect(); - // Resolver for group-by: after resolving through projections, substitute any right join key - // with its equivalent left join key. + // Resolver for group-by: after resolving through projections, substitute any + // build-side join key with its equivalent probe-side join key. let resolve_gby = |idx: usize| -> Option { let resolved = resolve_index(idx)?; Some(key_subst.get(&resolved).copied().unwrap_or(resolved)) }; - // Classify each group-by column as left-side or right-side. - // Left-side: resolves (with key substitution) to left schema. - // Right-side: resolves to right schema (any right column, not just join keys). + // Classify each group-by column: + // - Probe side (GroupBySide::Left): resolves (with key_subst) to probe range + // - Build side (GroupBySide::Right): resolves to build range let mut group_by_order: Vec = Vec::new(); for (expr, _alias) in group_by.expr() { - if expr_columns_in_side(expr.as_ref(), &resolve_gby, 0, left_field_count) { - group_by_order.push(GroupBySide::Left); + if expr_columns_in_side( + expr.as_ref(), + &resolve_gby, + probe_range_start, + probe_range_end, + ) { + group_by_order.push(GroupBySide::Left); // probe side } else if expr_columns_in_side( expr.as_ref(), &*resolve_index, - left_field_count, - usize::MAX, + build_range_start, + build_range_end, ) { - group_by_order.push(GroupBySide::Right); + group_by_order.push(GroupBySide::Right); // build side } else { - // Expression references neither side cleanly (e.g. spans both) return Ok(None); } } @@ -244,37 +274,53 @@ fn try_create_group_join( for agg_expr in agg_exec.aggr_expr() { let args = agg_expr.expressions(); if args.is_empty() { - // COUNT(*) style — treat as right side (no actual args to evaluate) + // COUNT(*) style — treat as build side (no actual args to evaluate) aggr_arg_sides.push(GroupBySide::Right); continue; } - let all_right = args.iter().all(|arg| { - expr_columns_in_side(arg.as_ref(), &*resolve_index, left_field_count, usize::MAX) + let all_build = args.iter().all(|arg| { + expr_columns_in_side( + arg.as_ref(), + &*resolve_index, + build_range_start, + build_range_end, + ) }); - let all_left = args.iter().all(|arg| { - expr_columns_in_side(arg.as_ref(), &*resolve_index, 0, left_field_count) + let all_probe = args.iter().all(|arg| { + expr_columns_in_side( + arg.as_ref(), + &*resolve_index, + probe_range_start, + probe_range_end, + ) }); - if all_right { - aggr_arg_sides.push(GroupBySide::Right); - } else if all_left { - aggr_arg_sides.push(GroupBySide::Left); + if all_build { + aggr_arg_sides.push(GroupBySide::Right); // build side + } else if all_probe { + aggr_arg_sides.push(GroupBySide::Left); // probe side } else { - // Args span both sides — cannot fuse return Ok(None); } } - // Remap group-by into left and right lists, preserving the group_by_order mapping + // Remap group-by into probe (left) and build (right) lists let mut left_gby: Vec<(Arc, String)> = Vec::new(); let mut right_gby: Vec<(Arc, String)> = Vec::new(); for ((expr, alias), &side) in group_by.expr().iter().zip(group_by_order.iter()) { match side { GroupBySide::Left => { - left_gby.push((remap_columns(expr, &resolve_gby, 0), alias.clone())); + // Probe side: remap with key_subst, subtract probe offset → 0-based in HashJoin.right schema + left_gby.push(( + remap_columns(expr, &resolve_gby, probe_range_start), + alias.clone(), + )); } GroupBySide::Right => { - right_gby - .push((remap_columns(expr, &*resolve_index, left_field_count), alias.clone())); + // Build side: remap, subtract build offset → 0-based in HashJoin.left schema + right_gby.push(( + remap_columns(expr, &*resolve_index, build_range_start), + alias.clone(), + )); } } } @@ -286,9 +332,9 @@ fn try_create_group_join( .zip(aggr_arg_sides.iter()) .map(|(expr, &side)| { let offset = if side == GroupBySide::Right { - left_field_count + build_range_start } else { - 0 + probe_range_start }; let remapped_args: Vec> = expr .expressions() @@ -305,11 +351,20 @@ fn try_create_group_join( let remapped_aggr_exprs: Vec<_> = remapped_aggr_exprs.into_iter().map(Arc::new).collect(); + // Swap inputs: GroupJoinExec.left (probe) = HashJoin.right, + // GroupJoinExec.right (build) = HashJoin.left. + // Also swap the ON keys accordingly. + let swapped_on: JoinOn = hash_join + .on() + .iter() + .map(|(l, r)| (Arc::clone(r), Arc::clone(l))) + .collect(); + GroupJoinExec::try_new_extended( *agg_exec.mode(), - Arc::clone(hash_join.left()), - Arc::clone(hash_join.right()), - hash_join.on().to_vec(), + Arc::clone(hash_join.right()), // probe side → GroupJoinExec.left + Arc::clone(hash_join.left()), // build side → GroupJoinExec.right + swapped_on, left_gby, right_gby, group_by_order, diff --git a/datafusion/physical-plan/src/joins/group_join.rs b/datafusion/physical-plan/src/joins/group_join.rs index 58804a6f63707..550bc180da4f1 100644 --- a/datafusion/physical-plan/src/joins/group_join.rs +++ b/datafusion/physical-plan/src/joins/group_join.rs @@ -44,8 +44,8 @@ use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::PhysicalExprRef; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::utils::evaluate_expressions_to_arrays; -use futures::future::poll_fn; use futures::StreamExt; +use futures::future::poll_fn; use crate::aggregates::group_values::{GroupValues, new_group_values}; use crate::aggregates::order::GroupOrdering; @@ -79,6 +79,9 @@ struct RightData { values: Vec, /// Evaluated right-side group-by columns (for group key construction) group_by_arrays: Vec, + /// Pre-evaluated right-side (build-side) aggregate argument arrays. + /// One Vec per build-side aggregate, computed once and reused. + build_agg_arrays: Vec>, } impl fmt::Debug for RightData { @@ -166,6 +169,7 @@ impl GroupJoinExec { /// - `group_by_order`: for each output group-by column, `Left` or `Right` /// (consumed left-to-right within each side's list) /// - `aggr_arg_sides`: for each aggregate expression, which input side provides args + #[expect(clippy::too_many_arguments)] pub fn try_new_extended( mode: AggregateMode, left: Arc, @@ -374,6 +378,14 @@ impl ExecutionPlan for GroupJoinExec { .iter() .map(|(e, _)| Arc::clone(e)) .collect(); + // Collect build-side aggregate argument expressions to pre-evaluate once + let build_agg_exprs: Vec>> = self + .aggr_expr + .iter() + .zip(self.aggr_arg_sides.iter()) + .filter(|&(_, side)| *side == GroupBySide::Right) + .map(|(agg, _)| agg.expressions()) + .collect(); let random_state = RandomState::with_seed(0); let ctx = Arc::clone(&context); self.right_fut.try_once(|| { @@ -381,6 +393,7 @@ impl ExecutionPlan for GroupJoinExec { right, on_right, right_group_by_exprs, + build_agg_exprs, random_state, ctx, )) @@ -403,8 +416,11 @@ impl ExecutionPlan for GroupJoinExec { .map(|(agg, &side)| (agg.expressions(), side)) .collect(); - let left_group_by_exprs: Vec = - self.left_group_by.iter().map(|(e, _)| Arc::clone(e)).collect(); + let left_group_by_exprs: Vec = self + .left_group_by + .iter() + .map(|(e, _)| Arc::clone(e)) + .collect(); let group_by_order = self.group_by_order.clone(); let mode = self.mode; @@ -535,12 +551,9 @@ impl GroupJoinStream { async fn build_right_side(&mut self) -> Result<()> { // Await the shared right-side future (built once across all left partitions). // OnceFut uses a poll-based API, so we bridge it via poll_fn. - let right_data: Arc = poll_fn(|cx| { - self.right_fut - .get(cx) - .map(|r| r.map(|arc| Arc::clone(arc))) - }) - .await?; + let right_data: Arc = + poll_fn(|cx| self.right_fut.get(cx).map(|r| r.map(Arc::clone))) + .await?; let batch_mem = get_record_batch_memory_size(&right_data.batch); self.reservation.try_grow(batch_mem)?; self.right_data = Some(right_data); @@ -582,21 +595,8 @@ impl GroupJoinStream { }) .collect::>()?; - // Pre-evaluate right-side aggregate arguments (right side is already built) - let right_agg_arrays: Vec> = self - .agg_args - .iter() - .filter(|(_, side)| *side == GroupBySide::Right) - .map(|(exprs, _)| { - exprs - .iter() - .map(|expr| { - expr.evaluate(&right_data.batch) - .and_then(|v| v.into_array(right_data.batch.num_rows())) - }) - .collect::>>() - }) - .collect::>()?; + // Use pre-computed build-side aggregate arrays from RightData + let right_agg_arrays = &right_data.build_agg_arrays; while let Some(current_offset) = offset { probe_indices_buf.clear(); @@ -721,7 +721,7 @@ impl GroupJoinStream { } fn emit_results(&mut self) -> Result { - if self.group_values.len() == 0 { + if self.group_values.is_empty() { return Ok(RecordBatch::new_empty(Arc::clone(&self.schema))); } let group_columns = self.group_values.emit(EmitTo::All)?; @@ -766,6 +766,7 @@ async fn collect_right_input( right: Arc, on_right: Vec, right_group_by_exprs: Vec, + build_agg_exprs: Vec>>, random_state: RandomState, context: Arc, ) -> Result> { @@ -793,6 +794,7 @@ async fn collect_right_input( batch: empty_batch, values: vec![], group_by_arrays: vec![], + build_agg_arrays: vec![], })); } @@ -831,11 +833,26 @@ async fn collect_right_input( // Evaluate right-side group-by columns on the concatenated batch. let group_by_arrays = evaluate_expressions_to_arrays(&right_group_by_exprs, &batch)?; + // Pre-evaluate build-side aggregate argument arrays once (shared across all probe batches). + let build_agg_arrays: Vec> = build_agg_exprs + .iter() + .map(|exprs| { + exprs + .iter() + .map(|expr| { + expr.evaluate(&batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) + .collect::>>() + }) + .collect::>()?; + Ok(Arc::new(RightData { map: Arc::new(Map::HashMap(hashmap)), batch, values, group_by_arrays, + build_agg_arrays, })) } diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index d1e729cfd2b17..a6ea2fcea7a9e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1340,7 +1340,7 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[] 02)--RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -03)----GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as t1_id], aggr=[] +03)----GroupJoinExec: mode=Partial, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as t1_id], aggr=[] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1395,7 +1395,7 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -04)------GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +04)------GroupJoinExec: mode=Partial, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1422,7 +1422,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 -07)------------GroupJoinExec: mode=Partial, on=[(t1_id@0, t2_id@0)], group_by=[t1_id@0 as alias1], aggr=[] +07)------------GroupJoinExec: mode=Partial, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 62649148bf058..3a84bf7a3437c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -74,19 +74,17 @@ physical_plan 03)----ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] 04)------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] -08)--------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] -10)------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] -12)----------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=csv, has_header=false -14)----------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -15)------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -16)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -17)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)--------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -19)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false -20)--------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -21)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +06)----------GroupJoinExec: mode=Partial, on=[(c_nationkey@3, n_nationkey@0)], group_by=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@5 as c_acctbal, c_phone@4 as c_phone, n_name@1 as n_name, c_address@2 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] +08)--------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] +10)------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=csv, has_header=false +12)------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +13)--------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +14)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false +15)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +16)----------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +17)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false +18)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index a31579eb1e09d..ab927bfdcf51f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -80,30 +80,27 @@ physical_plan 05)--------CoalescePartitionsExec 06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 07)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] -10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] -12)----------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -14)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -15)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -16)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -17)--------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -18)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -20)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -21)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -22)----------CoalescePartitionsExec -23)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -24)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -25)----------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -26)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -27)--------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -28)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -29)--------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -30)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -31)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -32)------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -33)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +08)--------------GroupJoinExec: mode=Partial, on=[(s_nationkey@3, n_nationkey@0)], group_by=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] +10)------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false +12)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +13)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +14)----------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +17)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +18)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +19)----------CoalescePartitionsExec +20)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +21)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +22)----------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +23)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +24)--------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +25)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false +26)--------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +27)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +28)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +29)------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +30)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index d5f82737baecd..cd00ad41e79ea 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -64,7 +64,7 @@ physical_plan 03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 05)--------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -06)----------GroupJoinExec: mode=Partial, on=[(l_orderkey@0, o_orderkey@0)], group_by=[l_shipmode@1 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] -07)------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false -09)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false +06)----------GroupJoinExec: mode=Partial, on=[(o_orderkey@0, l_orderkey@0)], group_by=[l_shipmode@1 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +07)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false +08)------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] +09)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index d5ff6724402ad..fe70536c3db06 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -130,19 +130,16 @@ physical_plan 29)----------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] 30)------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 31)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -33)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -34)--------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -35)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -36)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -39)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -40)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -41)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -42)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -43)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -45)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +32)----------------GroupJoinExec: mode=Partial, on=[(n_regionkey@2, r_regionkey@0)], group_by=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +33)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +34)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +35)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +36)------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +37)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +38)------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +39)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +40)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +41)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +42)------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +43)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index d854001f3cc4c..f7dfb76db82e5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -72,28 +72,25 @@ physical_plan 03)----ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] 04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] -08)--------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] -10)------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] -12)----------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] -14)--------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] -16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -18)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -19)--------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -20)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -21)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -22)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -23)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 -24)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -25)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -26)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -27)--------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -28)----------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -29)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +06)----------GroupJoinExec: mode=Partial, on=[(n_regionkey@3, r_regionkey@0)], group_by=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] +08)--------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] +10)------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] +14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false +16)--------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +17)----------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +18)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false +19)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +20)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +21)------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +22)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +23)--------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +24)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +25)------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +26)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false From 97dc899e53bafcb5c5aa3665d5bb5ea12e393888 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 18 Mar 2026 07:29:11 +0100 Subject: [PATCH 6/8] Extra --- .../physical-optimizer/src/group_join.rs | 31 +++- .../physical-plan/src/joins/group_join.rs | 171 +++++++++++++++--- .../sqllogictest/test_files/group_join.slt | 2 +- datafusion/sqllogictest/test_files/joins.slt | 6 +- 4 files changed, 175 insertions(+), 35 deletions(-) diff --git a/datafusion/physical-optimizer/src/group_join.rs b/datafusion/physical-optimizer/src/group_join.rs index f31b06fc78a5d..560054bd294f8 100644 --- a/datafusion/physical-optimizer/src/group_join.rs +++ b/datafusion/physical-optimizer/src/group_join.rs @@ -157,8 +157,9 @@ fn try_create_group_join( hash_join: &HashJoinExec, proj_exec: Option<&ProjectionExec>, ) -> Result> { - // Only support INNER joins - if *hash_join.join_type() != JoinType::Inner { + // Only support INNER and LEFT OUTER joins + let join_type = *hash_join.join_type(); + if join_type != JoinType::Inner && join_type != JoinType::Left { return Ok(None); } @@ -167,11 +168,15 @@ fn try_create_group_join( return Ok(None); } - // Only support CollectLeft mode — Partitioned mode has per-partition - // semantics that GroupJoinExec (which collects the entire build side) doesn't match. - if *hash_join.partition_mode() != PartitionMode::CollectLeft { - return Ok(None); - } + // Support CollectLeft and Partitioned modes. + // CollectLeft: build side collected once, shared across all probe partitions. + // Partitioned: both sides repartitioned by key, each partition builds independently. + let partition_mode = *hash_join.partition_mode(); + let partitioned = match partition_mode { + PartitionMode::CollectLeft => false, + PartitionMode::Partitioned => true, + _ => return Ok(None), + }; let group_by = agg_exec.group_expr(); if group_by.has_grouping_set() { @@ -269,6 +274,16 @@ fn try_create_group_join( } } + // For LEFT JOIN, all group-by columns must come from the probe side. + // Build-side group-by columns would be NULL for unmatched rows, which + // complicates group key handling. Q13 (the primary use case) has all + // group-by on the probe side, so this restriction is acceptable. + if join_type == JoinType::Left + && group_by_order.iter().any(|s| *s == GroupBySide::Right) + { + return Ok(None); + } + // Classify each aggregate's argument expressions: all must resolve to a single side. let mut aggr_arg_sides: Vec = Vec::new(); for agg_expr in agg_exec.aggr_expr() { @@ -370,6 +385,8 @@ fn try_create_group_join( group_by_order, remapped_aggr_exprs, aggr_arg_sides, + join_type, + partitioned, Arc::clone(&agg_exec.schema()), ) .map(Some) diff --git a/datafusion/physical-plan/src/joins/group_join.rs b/datafusion/physical-plan/src/joins/group_join.rs index 550bc180da4f1..1e0b4aaa4210a 100644 --- a/datafusion/physical-plan/src/joins/group_join.rs +++ b/datafusion/physical-plan/src/joins/group_join.rs @@ -30,7 +30,7 @@ use std::any::Any; use std::fmt; use std::sync::Arc; -use arrow::array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; +use arrow::array::{ArrayRef, BooleanArray, RecordBatch, UInt32Array, UInt64Array}; use arrow::compute; use arrow::compute::concat_batches; use arrow::datatypes::SchemaRef; @@ -38,6 +38,7 @@ use datafusion_common::hash_utils::{RandomState, create_hashes}; use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_common::utils::memory::get_record_batch_memory_size; use datafusion_common::{NullEquality, Result}; +use datafusion_expr::JoinType; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_expr::{EmitTo, GroupsAccumulator}; @@ -123,13 +124,17 @@ pub struct GroupJoinExec { aggr_expr: Vec>, /// For each aggr_expr, which input side provides the argument values aggr_arg_sides: Vec, + /// Join type (Inner or Left) + join_type: JoinType, + /// Whether to build per-partition (Partitioned mode) or shared (CollectLeft mode) + partitioned: bool, /// Output schema (matches the original AggregateExec's output) schema: SchemaRef, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cached plan properties cache: Arc, - /// Shared right-side hash table — built once, reused by all left partitions + /// Shared right-side hash table — built once, reused by all left partitions (CollectLeft mode only) right_fut: Arc>>, } @@ -157,6 +162,8 @@ impl GroupJoinExec { vec![GroupBySide::Left; n], aggr_expr, vec![GroupBySide::Right; a], + JoinType::Inner, + false, schema, ) } @@ -180,6 +187,8 @@ impl GroupJoinExec { group_by_order: Vec, aggr_expr: Vec>, aggr_arg_sides: Vec, + join_type: JoinType, + partitioned: bool, schema: SchemaRef, ) -> Result { let cache = Arc::new(Self::compute_properties( @@ -196,6 +205,8 @@ impl GroupJoinExec { group_by_order, aggr_expr, aggr_arg_sides, + join_type, + partitioned, schema, metrics: ExecutionPlanMetricsSet::new(), cache, @@ -261,8 +272,9 @@ impl DisplayAs for GroupJoinExec { .collect(); write!( f, - "GroupJoinExec: mode={:?}, on=[{}], group_by=[{}], aggr=[{}]", + "GroupJoinExec: mode={:?}, join_type={:?}, on=[{}], group_by=[{}], aggr=[{}]", self.mode, + self.join_type, on, group_by.join(", "), aggrs.join(", "), @@ -353,6 +365,8 @@ impl ExecutionPlan for GroupJoinExec { self.group_by_order.clone(), self.aggr_expr.clone(), self.aggr_arg_sides.clone(), + self.join_type, + self.partitioned, Arc::clone(&self.schema), )?)) } @@ -367,8 +381,7 @@ impl ExecutionPlan for GroupJoinExec { let left_stream = self.left.execute(partition, Arc::clone(&context))?; - // Build right-side hash table exactly once, shared across all left partitions. - // This mirrors HashJoinExec(CollectLeft) — the build side is computed once. + // Build right-side hash table. let right_fut: OnceFut> = { let right = Arc::clone(&self.right); let on_right: Vec = @@ -388,16 +401,32 @@ impl ExecutionPlan for GroupJoinExec { .collect(); let random_state = RandomState::with_seed(0); let ctx = Arc::clone(&context); - self.right_fut.try_once(|| { - Ok(collect_right_input( + if self.partitioned { + // Partitioned mode: each partition builds from its own right partition only. + let part = partition; + OnceFut::new(collect_right_input( right, + Some(part), on_right, right_group_by_exprs, build_agg_exprs, random_state, ctx, )) - })? + } else { + // CollectLeft mode: build once, shared across all left partitions. + self.right_fut.try_once(|| { + Ok(collect_right_input( + right, + None, // all partitions + on_right, + right_group_by_exprs, + build_agg_exprs, + random_state, + ctx, + )) + })? + } }; let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); @@ -440,12 +469,15 @@ impl ExecutionPlan for GroupJoinExec { let random_state = RandomState::with_seed(0); + let is_left_join = self.join_type == JoinType::Left; + let stream = GroupJoinStream { schema: Arc::clone(&schema), on_left, right_fut, left_stream: Some(left_stream), mode, + is_left_join, left_group_by_exprs, group_by_order, agg_args, @@ -494,6 +526,8 @@ struct GroupJoinStream { left_stream: Option, mode: AggregateMode, + /// Whether this is a LEFT OUTER JOIN (unmatched probe rows produce NULL build-side values) + is_left_join: bool, /// Group-by expressions evaluated against the LEFT (probe) input left_group_by_exprs: Vec, /// Interleave order for group-by output columns @@ -567,7 +601,25 @@ impl GroupJoinStream { ) })?; + // Pre-evaluate left-side group-by columns against the current left batch + let left_gby_arrays: Vec = self + .left_group_by_exprs + .iter() + .map(|expr| { + expr.evaluate(left_batch) + .and_then(|v| v.into_array(left_batch.num_rows())) + }) + .collect::>()?; + + // For LEFT JOIN with empty build side: all rows are unmatched if right_data.map.is_empty() { + if self.is_left_join { + self.emit_unmatched_left_rows( + left_batch, + &left_gby_arrays, + &BooleanArray::from(vec![false; left_batch.num_rows()]), + )?; + } return Ok(()); } @@ -585,19 +637,12 @@ impl GroupJoinStream { let mut build_indices_buf: Vec = Vec::new(); let mut offset: Option = Some((0, None)); - // Pre-evaluate left-side group-by columns against the current left batch - let left_gby_arrays: Vec = self - .left_group_by_exprs - .iter() - .map(|expr| { - expr.evaluate(left_batch) - .and_then(|v| v.into_array(left_batch.num_rows())) - }) - .collect::>()?; - // Use pre-computed build-side aggregate arrays from RightData let right_agg_arrays = &right_data.build_agg_arrays; + // For LEFT JOIN, track which probe rows had at least one match + let mut matched_probe = vec![false; left_batch.num_rows()]; + while let Some(current_offset) = offset { probe_indices_buf.clear(); build_indices_buf.clear(); @@ -639,6 +684,11 @@ impl GroupJoinStream { continue; } + // Mark matched probe rows + for &idx in probe_indices.values() { + matched_probe[idx as usize] = true; + } + // Build the ordered group-by array list by interleaving left and right let left_taken: Vec = left_gby_arrays .iter() @@ -675,13 +725,11 @@ impl GroupJoinStream { // Feed matched values to accumulators (left args taken at probe_indices, right at build_indices) let mut right_agg_idx = 0; - for (acc_idx, (acc, (arg_exprs, side))) in self + for (acc, (arg_exprs, side)) in self .accumulators .iter_mut() .zip(self.agg_args.iter()) - .enumerate() { - let _ = acc_idx; let values: Vec = match side { GroupBySide::Left => arg_exprs .iter() @@ -717,6 +765,75 @@ impl GroupJoinStream { offset = next_offset; } + // For LEFT JOIN: emit group entries for unmatched probe rows. + // Accumulators are NOT updated for these rows, so they keep default values + // (0 for COUNT, NULL for SUM, etc.) which is correct LEFT JOIN semantics. + if self.is_left_join { + let matched_bitmap = BooleanArray::from(matched_probe); + self.emit_unmatched_left_rows(left_batch, &left_gby_arrays, &matched_bitmap)?; + } + + Ok(()) + } + + /// For LEFT JOIN: intern group keys for probe rows that had no matches. + /// Accumulators are not updated, so they retain default values (correct for LEFT JOIN). + fn emit_unmatched_left_rows( + &mut self, + _left_batch: &RecordBatch, + left_gby_arrays: &[ArrayRef], + matched_bitmap: &BooleanArray, + ) -> Result<()> { + let unmatched: Vec = matched_bitmap + .iter() + .enumerate() + .filter(|(_, matched)| !matched.unwrap_or(false)) + .map(|(i, _)| i as u32) + .collect(); + + if unmatched.is_empty() { + return Ok(()); + } + + let unmatched_indices = UInt32Array::from(unmatched); + + // Build group keys: probe-side taken at unmatched indices, build-side filled with NULLs + let unmatched_left_gby: Vec = left_gby_arrays + .iter() + .map(|arr| compute::take(arr.as_ref(), &unmatched_indices, None)) + .collect::>()?; + + let right_data = self.right_data.as_ref().unwrap(); + let unmatched_right_gby: Vec = right_data + .group_by_arrays + .iter() + .map(|arr| { + arrow::array::new_null_array(arr.data_type(), unmatched_indices.len()) + }) + .collect(); + + let mut left_i = 0; + let mut right_i = 0; + let group_values: Vec = self + .group_by_order + .iter() + .map(|side| match side { + GroupBySide::Left => { + let v = Arc::clone(&unmatched_left_gby[left_i]); + left_i += 1; + v + } + GroupBySide::Right => { + let v = Arc::clone(&unmatched_right_gby[right_i]); + right_i += 1; + v + } + }) + .collect(); + + self.group_values + .intern(&group_values, &mut self.current_group_indices)?; + Ok(()) } @@ -764,18 +881,24 @@ impl GroupJoinStream { /// via `OnceAsync`, so the right side is collected exactly once. async fn collect_right_input( right: Arc, + single_partition: Option, on_right: Vec, right_group_by_exprs: Vec, build_agg_exprs: Vec>>, random_state: RandomState, context: Arc, ) -> Result> { - let num_partitions = right.output_partitioning().partition_count(); let schema = right.schema(); - // Execute all right partitions and collect all batches. + // Execute right partitions and collect batches. + // single_partition=Some(p) collects only partition p (Partitioned mode). + // single_partition=None collects all partitions (CollectLeft mode). let mut all_batches: Vec = Vec::new(); - for p in 0..num_partitions { + let partitions: Vec = match single_partition { + Some(p) => vec![p], + None => (0..right.output_partitioning().partition_count()).collect(), + }; + for p in partitions { let mut stream = right.execute(p, Arc::clone(&context))?; while let Some(batch) = stream.next().await { let batch = batch?; diff --git a/datafusion/sqllogictest/test_files/group_join.slt b/datafusion/sqllogictest/test_files/group_join.slt index 7ae72b9284e72..2da3743329646 100644 --- a/datafusion/sqllogictest/test_files/group_join.slt +++ b/datafusion/sqllogictest/test_files/group_join.slt @@ -54,7 +54,7 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[customer_id@0 as customer_id], aggr=[sum(li.amount)] 02)--RepartitionExec: partitioning=Hash([customer_id@0], 4), input_partitions=1 -03)----GroupJoinExec: mode=Partial, on=[(order_id@0, order_id@0)], group_by=[customer_id@1 as customer_id], aggr=[sum(li.amount)] +03)----GroupJoinExec: mode=Partial, join_type=Inner, on=[(order_id@0, order_id@0)], group_by=[customer_id@1 as customer_id], aggr=[sum(li.amount)] 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index a6ea2fcea7a9e..a57c6221fb3d7 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1340,7 +1340,7 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[] 02)--RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -03)----GroupJoinExec: mode=Partial, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as t1_id], aggr=[] +03)----GroupJoinExec: mode=Partial, join_type=Inner, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as t1_id], aggr=[] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1395,7 +1395,7 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -04)------GroupJoinExec: mode=Partial, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +04)------GroupJoinExec: mode=Partial, join_type=Inner, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1422,7 +1422,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 -07)------------GroupJoinExec: mode=Partial, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as alias1], aggr=[] +07)------------GroupJoinExec: mode=Partial, join_type=Inner, on=[(t2_id@0, t1_id@0)], group_by=[t1_id@0 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 From 2ae8c3e8f68c2de7c4c61ffca33291e1c3bdfbf6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 18 Mar 2026 08:10:16 +0100 Subject: [PATCH 7/8] Extra --- .../physical-optimizer/src/group_join.rs | 11 ++++++++++- .../repartition_subset_satisfaction.slt | 4 ++-- .../tpch/plans/q1.slt.part0008204857.temp | 18 ++++++++++++++++++ .../test_files/tpch/plans/q10.slt.part | 2 +- .../test_files/tpch/plans/q11.slt.part | 2 +- .../test_files/tpch/plans/q12.slt.part | 2 +- .../test_files/tpch/plans/q2.slt.part | 2 +- .../test_files/tpch/plans/q3.slt.part | 2 +- .../test_files/tpch/plans/q5.slt.part | 2 +- 9 files changed, 36 insertions(+), 9 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part0008204857.temp diff --git a/datafusion/physical-optimizer/src/group_join.rs b/datafusion/physical-optimizer/src/group_join.rs index 560054bd294f8..67d091f82e4b6 100644 --- a/datafusion/physical-optimizer/src/group_join.rs +++ b/datafusion/physical-optimizer/src/group_join.rs @@ -30,7 +30,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::aggregates::{AggregateExec, AggregateMode}; use datafusion_physical_plan::joins::HashJoinExec; use datafusion_physical_plan::joins::group_join::{GroupBySide, GroupJoinExec}; use datafusion_physical_plan::joins::{JoinOn, PartitionMode}; @@ -178,6 +178,15 @@ fn try_create_group_join( _ => return Ok(None), }; + // For Partitioned mode, a group key may appear in multiple partitions. + // We can only fuse if the aggregate is Partial (with a Final stage above + // that merges across partitions). SinglePartitioned/Single modes assume + // each partition produces complete results, which GroupJoinExec can't + // guarantee when group-by keys differ from the join/partition key. + if partitioned && *agg_exec.mode() != AggregateMode::Partial { + return Ok(None); + } + let group_by = agg_exec.group_expr(); if group_by.has_grouping_set() { return Ok(None); diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index d2d58a771a99d..95e8f9b314c4f 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -376,7 +376,7 @@ physical_plan 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)] 09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 -10)------------------GroupJoinExec: mode=Partial, on=[(d_dkey@1, f_dkey@2)], group_by=[f_dkey@1 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@0) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@0 as env], aggr=[max(j.value)] +10)------------------GroupJoinExec: mode=Partial, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], group_by=[f_dkey@1 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@0) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@0 as env], aggr=[max(j.value)] 11)--------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] 12)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] 13)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet @@ -471,7 +471,7 @@ physical_plan 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)] 09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3 -10)------------------GroupJoinExec: mode=Partial, on=[(d_dkey@1, f_dkey@2)], group_by=[f_dkey@1 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@0) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@0 as env], aggr=[max(j.value)] +10)------------------GroupJoinExec: mode=Partial, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], group_by=[f_dkey@1 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@0) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@0 as env], aggr=[max(j.value)] 11)--------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] 12)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] 13)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part0008204857.temp b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part0008204857.temp new file mode 100644 index 0000000000000..837a347bbb4fd --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part0008204857.temp @@ -0,0 +1,18 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 3a84bf7a3437c..06de03a140b68 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -74,7 +74,7 @@ physical_plan 03)----ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] 04)------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 -06)----------GroupJoinExec: mode=Partial, on=[(c_nationkey@3, n_nationkey@0)], group_by=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@5 as c_acctbal, c_phone@4 as c_phone, n_name@1 as n_name, c_address@2 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------GroupJoinExec: mode=Partial, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], group_by=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@5 as c_acctbal, c_phone@4 as c_phone, n_name@1 as n_name, c_address@2 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] 08)--------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index ab927bfdcf51f..7d40be1c7123e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -80,7 +80,7 @@ physical_plan 05)--------CoalescePartitionsExec 06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 07)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -08)--------------GroupJoinExec: mode=Partial, on=[(s_nationkey@3, n_nationkey@0)], group_by=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +08)--------------GroupJoinExec: mode=Partial, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], group_by=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] 10)------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index cd00ad41e79ea..fa73602f33e4a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -64,7 +64,7 @@ physical_plan 03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 05)--------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -06)----------GroupJoinExec: mode=Partial, on=[(o_orderkey@0, l_orderkey@0)], group_by=[l_shipmode@1 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +06)----------GroupJoinExec: mode=Partial, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], group_by=[l_shipmode@1 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 07)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false 08)------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] 09)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index fe70536c3db06..22f64b0f94bd9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -130,7 +130,7 @@ physical_plan 29)----------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] 30)------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 31)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)----------------GroupJoinExec: mode=Partial, on=[(n_regionkey@2, r_regionkey@0)], group_by=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +32)----------------GroupJoinExec: mode=Partial, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], group_by=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 33)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] 34)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 35)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index e08fe5fe7671a..487bee3a0ebd6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -63,7 +63,7 @@ physical_plan 03)----ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 -06)----------GroupJoinExec: mode=Partial, on=[(o_orderkey@0, l_orderkey@0)], group_by=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------GroupJoinExec: mode=Partial, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], group_by=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 08)--------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 09)----------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index f7dfb76db82e5..65e9f39ec8a77 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -72,7 +72,7 @@ physical_plan 03)----ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] 04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 -06)----------GroupJoinExec: mode=Partial, on=[(n_regionkey@3, r_regionkey@0)], group_by=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------GroupJoinExec: mode=Partial, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], group_by=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] 08)--------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] From 48e095c66f640bcc7f18cc56ef699215183c628c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 18 Mar 2026 10:37:58 +0100 Subject: [PATCH 8/8] perf: GroupJoin efficiency improvements and cleanup - Skip all work for inner joins with empty build side (avoid evaluating group-by arrays) - Make matched_probe allocation and marking conditional on LEFT JOIN only - Persist probe/build index buffers across batches instead of re-allocating - Use compute::filter instead of index materialization in emit_unmatched_left_rows - Extract interleave_arrays helper to deduplicate 3 copies of the pattern - Remove unused _left_batch parameter from emit_unmatched_left_rows - Wire up baseline_metrics to record output rows Co-Authored-By: Claude Opus 4.6 (1M context) --- .../physical-plan/src/joins/group_join.rs | 155 +++++++++--------- 1 file changed, 81 insertions(+), 74 deletions(-) diff --git a/datafusion/physical-plan/src/joins/group_join.rs b/datafusion/physical-plan/src/joins/group_join.rs index 1e0b4aaa4210a..c271a25ffde14 100644 --- a/datafusion/physical-plan/src/joins/group_join.rs +++ b/datafusion/physical-plan/src/joins/group_join.rs @@ -491,6 +491,8 @@ impl ExecutionPlan for GroupJoinExec { reservation, hashes_buffer: Vec::new(), random_state, + probe_indices_buf: Vec::new(), + build_indices_buf: Vec::new(), }; Ok(Box::pin(RecordBatchStreamAdapter::new( @@ -539,7 +541,6 @@ struct GroupJoinStream { current_group_indices: Vec, batch_size: usize, - #[expect(dead_code)] baseline_metrics: BaselineMetrics, state: GroupJoinStreamState, /// Resolved right-side data (set after BuildRight state completes) @@ -547,6 +548,9 @@ struct GroupJoinStream { reservation: MemoryReservation, hashes_buffer: Vec, random_state: RandomState, + /// Reusable buffers for probe/build indices (avoids per-batch allocation) + probe_indices_buf: Vec, + build_indices_buf: Vec, } impl GroupJoinStream { @@ -573,6 +577,7 @@ impl GroupJoinStream { let result = self.emit_results()?; self.state = GroupJoinStreamState::Done; if result.num_rows() > 0 { + self.baseline_metrics.record_output(result.num_rows()); return Ok(Some(result)); } return Ok(None); @@ -601,6 +606,11 @@ impl GroupJoinStream { ) })?; + // For inner joins with empty build side, skip all work + if right_data.map.is_empty() && !self.is_left_join { + return Ok(()); + } + // Pre-evaluate left-side group-by columns against the current left batch let left_gby_arrays: Vec = self .left_group_by_exprs @@ -613,13 +623,10 @@ impl GroupJoinStream { // For LEFT JOIN with empty build side: all rows are unmatched if right_data.map.is_empty() { - if self.is_left_join { - self.emit_unmatched_left_rows( - left_batch, - &left_gby_arrays, - &BooleanArray::from(vec![false; left_batch.num_rows()]), - )?; - } + self.emit_unmatched_left_rows( + &left_gby_arrays, + &BooleanArray::from(vec![false; left_batch.num_rows()]), + )?; return Ok(()); } @@ -633,19 +640,23 @@ impl GroupJoinStream { &mut self.hashes_buffer, )?; - let mut probe_indices_buf: Vec = Vec::new(); - let mut build_indices_buf: Vec = Vec::new(); + self.probe_indices_buf.clear(); + self.build_indices_buf.clear(); let mut offset: Option = Some((0, None)); // Use pre-computed build-side aggregate arrays from RightData let right_agg_arrays = &right_data.build_agg_arrays; // For LEFT JOIN, track which probe rows had at least one match - let mut matched_probe = vec![false; left_batch.num_rows()]; + let mut matched_probe = if self.is_left_join { + vec![false; left_batch.num_rows()] + } else { + vec![] + }; while let Some(current_offset) = offset { - probe_indices_buf.clear(); - build_indices_buf.clear(); + self.probe_indices_buf.clear(); + self.build_indices_buf.clear(); let (build_indices, probe_indices, next_offset) = match right_data.map.as_ref() { @@ -657,20 +668,24 @@ impl GroupJoinStream { &self.hashes_buffer, self.batch_size, current_offset, - &mut probe_indices_buf, - &mut build_indices_buf, + &mut self.probe_indices_buf, + &mut self.build_indices_buf, )?, Map::ArrayMap(array_map) => { let next = array_map.get_matched_indices_with_limit_offset( &left_key_values, self.batch_size, current_offset, - &mut probe_indices_buf, - &mut build_indices_buf, + &mut self.probe_indices_buf, + &mut self.build_indices_buf, )?; ( - UInt64Array::from(std::mem::take(&mut build_indices_buf)), - UInt32Array::from(std::mem::take(&mut probe_indices_buf)), + UInt64Array::from(std::mem::take( + &mut self.build_indices_buf, + )), + UInt32Array::from(std::mem::take( + &mut self.probe_indices_buf, + )), next, ) } @@ -684,9 +699,11 @@ impl GroupJoinStream { continue; } - // Mark matched probe rows - for &idx in probe_indices.values() { - matched_probe[idx as usize] = true; + // Mark matched probe rows (only for LEFT JOIN) + if self.is_left_join { + for &idx in probe_indices.values() { + matched_probe[idx as usize] = true; + } } // Build the ordered group-by array list by interleaving left and right @@ -700,24 +717,8 @@ impl GroupJoinStream { .map(|arr| compute::take(arr.as_ref(), &build_indices, None)) .collect::>()?; - let mut left_i = 0; - let mut right_i = 0; - let matched_group_values: Vec = self - .group_by_order - .iter() - .map(|side| match side { - GroupBySide::Left => { - let v = Arc::clone(&left_taken[left_i]); - left_i += 1; - v - } - GroupBySide::Right => { - let v = Arc::clone(&right_taken[right_i]); - right_i += 1; - v - } - }) - .collect(); + let matched_group_values = + interleave_arrays(&self.group_by_order, &left_taken, &right_taken); self.group_values .intern(&matched_group_values, &mut self.current_group_indices)?; @@ -770,7 +771,7 @@ impl GroupJoinStream { // (0 for COUNT, NULL for SUM, etc.) which is correct LEFT JOIN semantics. if self.is_left_join { let matched_bitmap = BooleanArray::from(matched_probe); - self.emit_unmatched_left_rows(left_batch, &left_gby_arrays, &matched_bitmap)?; + self.emit_unmatched_left_rows(&left_gby_arrays, &matched_bitmap)?; } Ok(()) @@ -780,56 +781,37 @@ impl GroupJoinStream { /// Accumulators are not updated, so they retain default values (correct for LEFT JOIN). fn emit_unmatched_left_rows( &mut self, - _left_batch: &RecordBatch, left_gby_arrays: &[ArrayRef], matched_bitmap: &BooleanArray, ) -> Result<()> { - let unmatched: Vec = matched_bitmap - .iter() - .enumerate() - .filter(|(_, matched)| !matched.unwrap_or(false)) - .map(|(i, _)| i as u32) - .collect(); + // Invert the bitmap to get unmatched rows, then filter + let unmatched_bitmap = compute::not(matched_bitmap)?; - if unmatched.is_empty() { + // Check if there are any unmatched rows + if unmatched_bitmap.true_count() == 0 { return Ok(()); } - let unmatched_indices = UInt32Array::from(unmatched); + let num_unmatched = unmatched_bitmap.true_count(); - // Build group keys: probe-side taken at unmatched indices, build-side filled with NULLs + // Build group keys: probe-side filtered by unmatched, build-side filled with NULLs let unmatched_left_gby: Vec = left_gby_arrays .iter() - .map(|arr| compute::take(arr.as_ref(), &unmatched_indices, None)) - .collect::>()?; + .map(|arr| { + compute::filter(arr.as_ref(), &unmatched_bitmap) + .map_err(Into::into) + }) + .collect::>()?; let right_data = self.right_data.as_ref().unwrap(); let unmatched_right_gby: Vec = right_data .group_by_arrays .iter() - .map(|arr| { - arrow::array::new_null_array(arr.data_type(), unmatched_indices.len()) - }) + .map(|arr| arrow::array::new_null_array(arr.data_type(), num_unmatched)) .collect(); - let mut left_i = 0; - let mut right_i = 0; - let group_values: Vec = self - .group_by_order - .iter() - .map(|side| match side { - GroupBySide::Left => { - let v = Arc::clone(&unmatched_left_gby[left_i]); - left_i += 1; - v - } - GroupBySide::Right => { - let v = Arc::clone(&unmatched_right_gby[right_i]); - right_i += 1; - v - } - }) - .collect(); + let group_values = + interleave_arrays(&self.group_by_order, &unmatched_left_gby, &unmatched_right_gby); self.group_values .intern(&group_values, &mut self.current_group_indices)?; @@ -870,6 +852,31 @@ impl GroupJoinStream { } } +/// Interleave left and right arrays according to the group-by order. +fn interleave_arrays( + group_by_order: &[GroupBySide], + left_arrays: &[ArrayRef], + right_arrays: &[ArrayRef], +) -> Vec { + let mut left_i = 0; + let mut right_i = 0; + group_by_order + .iter() + .map(|side| match side { + GroupBySide::Left => { + let v = Arc::clone(&left_arrays[left_i]); + left_i += 1; + v + } + GroupBySide::Right => { + let v = Arc::clone(&right_arrays[right_i]); + right_i += 1; + v + } + }) + .collect() +} + /// Collects all right-side (build) partitions into a single `RightData`. /// /// Mirrors the pattern used by `HashJoinExec(CollectLeft)`: