From 0595d6323338ac48802e39f00c989d6fb362278e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 26 Nov 2025 20:10:49 +0100 Subject: [PATCH 01/14] Refactor join macro to function --- .../physical-plan/src/joins/join_hash_map.rs | 117 ++++++++++-------- 1 file changed, 64 insertions(+), 53 deletions(-) diff --git a/datafusion/physical-plan/src/joins/join_hash_map.rs b/datafusion/physical-plan/src/joins/join_hash_map.rs index bdd4bfeeb0fb..13b6692e1fdf 100644 --- a/datafusion/physical-plan/src/joins/join_hash_map.rs +++ b/datafusion/physical-plan/src/joins/join_hash_map.rs @@ -254,39 +254,49 @@ impl JoinHashMapType for JoinHashMapU64 { // Type of offsets for obtaining indices from JoinHashMap. pub(crate) type JoinHashMapOffset = (usize, Option); -// Macro for traversing chained values with limit. -// Early returns in case of reaching output tuples limit. -macro_rules! chain_traverse { - ( - $input_indices:ident, $match_indices:ident, - $hash_values:ident, $next_chain:ident, - $input_idx:ident, $chain_idx:ident, $remaining_output:ident, $one:ident, $zero:ident - ) => {{ - // now `one` and `zero` are in scope from the outer function - let mut match_row_idx = $chain_idx - $one; - loop { - $match_indices.push(match_row_idx.into()); - $input_indices.push($input_idx as u32); - $remaining_output -= 1; - - let next = $next_chain[match_row_idx.into() as usize]; - - if $remaining_output == 0 { - // we compare against `zero` (of type T) here too - let next_offset = if $input_idx == $hash_values.len() - 1 && next == $zero - { - None - } else { - Some(($input_idx, Some(next.into()))) - }; - return ($input_indices, $match_indices, next_offset); - } - if next == $zero { - break; - } - match_row_idx = next - $one; +/// Traverses the chain of matching indices, collecting results up to the remaining limit. +/// Returns `Some(offset)` if the limit was reached and there are more results to process, +/// or `None` if the chain was fully traversed. +#[inline(always)] +fn traverse_chain( + next_chain: &[T], + input_idx: usize, + start_chain_idx: T, + remaining: &mut usize, + input_indices: &mut Vec, + match_indices: &mut Vec, + is_last_input: bool, +) -> Option +where + T: Copy + TryFrom + PartialOrd + Into + Sub, + >::Error: Debug, +{ + let zero = T::try_from(0).unwrap(); + let one = T::try_from(1).unwrap(); + let mut match_row_idx = start_chain_idx - one; + + loop { + match_indices.push(match_row_idx.into()); + input_indices.push(input_idx as u32); + *remaining -= 1; + + let next = next_chain[match_row_idx.into() as usize]; + + if *remaining == 0 { + // Limit reached - return offset for next call + return if is_last_input && next == zero { + // Finished processing the last input row + None + } else { + Some((input_idx, Some(next.into()))) + }; + } + if next == zero { + // End of chain + return None; } - }}; + match_row_idx = next - one; + } } pub fn update_from_iter<'a, T>( @@ -383,7 +393,6 @@ where { let mut input_indices = Vec::with_capacity(limit); let mut match_indices = Vec::with_capacity(limit); - let zero = T::try_from(0).unwrap(); let one = T::try_from(1).unwrap(); // Check if hashmap consists of unique values @@ -409,7 +418,7 @@ where // Calculate initial `hash_values` index before iterating let to_skip = match offset { - // None `initial_next_idx` indicates that `initial_idx` processing has'n been started + // None `initial_next_idx` indicates that `initial_idx` processing hasn't been started (idx, None) => idx, // Zero `initial_next_idx` indicates that `initial_idx` has been processed during // previous iteration, and it should be skipped @@ -418,38 +427,40 @@ where // to start with the next index (idx, Some(next_idx)) => { let next_idx: T = T::try_from(next_idx as usize).unwrap(); - chain_traverse!( - input_indices, - match_indices, - hash_values, + let is_last = idx == hash_values.len() - 1; + if let Some(next_offset) = traverse_chain( next_chain, idx, next_idx, - remaining_output, - one, - zero - ); + &mut remaining_output, + &mut input_indices, + &mut match_indices, + is_last, + ) { + return (input_indices, match_indices, Some(next_offset)); + } idx + 1 } }; - let mut row_idx = to_skip; - for &hash in &hash_values[to_skip..] { + let hash_values_len = hash_values.len(); + for (i, &hash) in hash_values[to_skip..].iter().enumerate() { + let row_idx = to_skip + i; if let Some((_, idx)) = map.find(hash, |(h, _)| hash == *h) { let idx: T = *idx; - chain_traverse!( - input_indices, - match_indices, - hash_values, + let is_last = row_idx == hash_values_len - 1; + if let Some(next_offset) = traverse_chain( next_chain, row_idx, idx, - remaining_output, - one, - zero - ); + &mut remaining_output, + &mut input_indices, + &mut match_indices, + is_last, + ) { + return (input_indices, match_indices, Some(next_offset)); + } } - row_idx += 1; } (input_indices, match_indices, None) } From 246f7dc5564e0cc81a73dc8b2ca9bf6b1243eddf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Wed, 26 Nov 2025 20:16:48 +0100 Subject: [PATCH 02/14] Minor opt --- datafusion/physical-plan/src/joins/join_hash_map.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/joins/join_hash_map.rs b/datafusion/physical-plan/src/joins/join_hash_map.rs index 13b6692e1fdf..0adc16450961 100644 --- a/datafusion/physical-plan/src/joins/join_hash_map.rs +++ b/datafusion/physical-plan/src/joins/join_hash_map.rs @@ -22,6 +22,7 @@ use std::fmt::{self, Debug}; use std::ops::Sub; +use arrow::datatypes::ArrowNativeType; use hashbrown::hash_table::Entry::{Occupied, Vacant}; use hashbrown::HashTable; @@ -270,9 +271,10 @@ fn traverse_chain( where T: Copy + TryFrom + PartialOrd + Into + Sub, >::Error: Debug, + T: ArrowNativeType, { - let zero = T::try_from(0).unwrap(); - let one = T::try_from(1).unwrap(); + let zero = T::usize_as(0); + let one = T::usize_as(1); let mut match_row_idx = start_chain_idx - one; loop { @@ -390,6 +392,7 @@ pub fn get_matched_indices_with_limit_offset( where T: Copy + TryFrom + PartialOrd + Into + Sub, >::Error: Debug, + T: ArrowNativeType, { let mut input_indices = Vec::with_capacity(limit); let mut match_indices = Vec::with_capacity(limit); @@ -426,7 +429,7 @@ where // Otherwise, process remaining `initial_idx` matches by traversing `next_chain`, // to start with the next index (idx, Some(next_idx)) => { - let next_idx: T = T::try_from(next_idx as usize).unwrap(); + let next_idx: T = T::usize_as(next_idx as usize); let is_last = idx == hash_values.len() - 1; if let Some(next_offset) = traverse_chain( next_chain, From 150977da862f076728967d7330b31ad9a04de0ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 27 Nov 2025 22:36:11 +0100 Subject: [PATCH 03/14] Move coalesce batches to hash join --- .../src/coalesce_batches.rs | 4 +- .../physical-plan/src/joins/hash_join/exec.rs | 61 +- .../src/joins/hash_join/stream.rs | 110 ++-- datafusion/sqllogictest/test_files/cte.slt | 7 +- .../dynamic_filter_pushdown_config.slt | 42 +- .../sqllogictest/test_files/explain_tree.slt | 232 +++---- .../sqllogictest/test_files/group_by.slt | 47 +- .../sqllogictest/test_files/join.slt.part | 84 ++- .../join_disable_repartition_joins.slt | 20 +- .../test_files/join_is_not_distinct_from.slt | 93 ++- datafusion/sqllogictest/test_files/joins.slt | 608 ++++++++---------- .../sqllogictest/test_files/predicates.slt | 45 +- .../test_files/push_down_filter.slt | 9 +- .../test_files/sort_merge_join.slt | 2 +- .../test_files/spark/bitwise/bit_count.slt | 2 +- .../test_files/spark/math/csc.slt | 2 +- .../sqllogictest/test_files/subquery.slt | 103 ++- .../test_files/tpch/plans/q10.slt.part | 43 +- .../test_files/tpch/plans/q11.slt.part | 70 +- .../test_files/tpch/plans/q12.slt.part | 17 +- .../test_files/tpch/plans/q13.slt.part | 17 +- .../test_files/tpch/plans/q14.slt.part | 17 +- .../test_files/tpch/plans/q15.slt.part | 44 +- .../test_files/tpch/plans/q16.slt.part | 34 +- .../test_files/tpch/plans/q17.slt.part | 32 +- .../test_files/tpch/plans/q18.slt.part | 41 +- .../test_files/tpch/plans/q19.slt.part | 21 +- .../test_files/tpch/plans/q2.slt.part | 128 ++-- .../test_files/tpch/plans/q20.slt.part | 68 +- .../test_files/tpch/plans/q21.slt.part | 69 +- .../test_files/tpch/plans/q22.slt.part | 31 +- .../test_files/tpch/plans/q3.slt.part | 34 +- .../test_files/tpch/plans/q4.slt.part | 19 +- .../test_files/tpch/plans/q5.slt.part | 69 +- .../test_files/tpch/plans/q7.slt.part | 73 +-- .../test_files/tpch/plans/q8.slt.part | 97 ++- .../test_files/tpch/plans/q9.slt.part | 67 +- datafusion/sqllogictest/test_files/union.slt | 90 ++- 38 files changed, 1206 insertions(+), 1346 deletions(-) diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 61e4c0e7f180..a0a308921322 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -58,9 +58,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { let target_batch_size = config.execution.batch_size; plan.transform_up(|plan| { let plan_any = plan.as_any(); - let wrap_in_coalesce = plan_any.downcast_ref::().is_some() - // Don't need to add CoalesceBatchesExec after a round robin RepartitionExec - || plan_any + let wrap_in_coalesce = plan_any .downcast_ref::() .map(|repart_exec| { !matches!( diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 680e4368ae70..aba4a3a42abc 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -1938,7 +1938,12 @@ mod tests { div_ceil(9, batch_size) }; - assert_eq!(batches.len(), expected_batch_count); + // With batch coalescing, we may have fewer batches than expected + assert!( + batches.len() <= expected_batch_count, + "expected at most {expected_batch_count} batches, got {}", + batches.len() + ); // Inner join output is expected to preserve both inputs order allow_duplicates! { @@ -2018,7 +2023,12 @@ mod tests { div_ceil(9, batch_size) }; - assert_eq!(batches.len(), expected_batch_count); + // With batch coalescing, we may have fewer batches than expected + assert!( + batches.len() <= expected_batch_count, + "expected at most {expected_batch_count} batches, got {}", + batches.len() + ); // Inner join output is expected to preserve both inputs order allow_duplicates! { @@ -2154,7 +2164,12 @@ mod tests { // and filtered later. div_ceil(6, batch_size) }; - assert_eq!(batches.len(), expected_batch_count); + // With batch coalescing, we may have fewer batches than expected + assert!( + batches.len() <= expected_batch_count, + "expected at most {expected_batch_count} batches, got {}", + batches.len() + ); // Inner join output is expected to preserve both inputs order allow_duplicates! { @@ -2179,7 +2194,12 @@ mod tests { // and filtered later. div_ceil(3, batch_size) }; - assert_eq!(batches.len(), expected_batch_count); + // With batch coalescing, we may have fewer batches than expected + assert!( + batches.len() <= expected_batch_count, + "expected at most {expected_batch_count} batches, got {}", + batches.len() + ); // Inner join output is expected to preserve both inputs order allow_duplicates! { @@ -4199,10 +4219,11 @@ mod tests { } _ => div_ceil(expected_resultset_records, batch_size) + 1, }; - assert_eq!( - batches.len(), - expected_batch_count, - "expected {expected_batch_count} output batches for {join_type} join with batch_size = {batch_size}" + // With batch coalescing, we may have fewer batches than expected + assert!( + batches.len() <= expected_batch_count, + "expected at most {expected_batch_count} output batches for {join_type} join with batch_size = {batch_size}, got {}", + batches.len() ); let expected = match join_type { @@ -4212,7 +4233,17 @@ mod tests { JoinType::LeftAnti => left_empty.to_vec(), _ => common_result.to_vec(), }; - assert_batches_eq!(expected, &batches); + // For anti joins with empty results, we may get zero batches + // (with coalescing) instead of one empty batch with schema + if batches.is_empty() { + // Verify this is an expected empty result case + assert!( + matches!(join_type, JoinType::RightAnti | JoinType::LeftAnti), + "Unexpected empty result for {join_type} join" + ); + } else { + assert_batches_eq!(expected, &batches); + } } } } @@ -4475,9 +4506,15 @@ mod tests { assert_join_metrics!(metrics, 0); - let expected_null_neq = - ["+----+----+", "| n1 | n2 |", "+----+----+", "+----+----+"]; - assert_batches_eq!(expected_null_neq, &batches_null_neq); + // With batch coalescing, empty results may not emit any batches + // Check that either we have no batches, or an empty batch with proper schema + if batches_null_neq.is_empty() { + // This is fine - no output rows + } else { + let expected_null_neq = + ["+----+----+", "| n1 | n2 |", "+----+----+", "+----+----+"]; + assert_batches_eq!(expected_null_neq, &batches_null_neq); + } Ok(()) } diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index a50a6551db4d..df6d262d110d 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -45,6 +45,7 @@ use crate::{ }; use arrow::array::{Array, ArrayRef, UInt32Array, UInt64Array}; +use arrow::compute::BatchCoalescer; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{ @@ -213,9 +214,12 @@ pub(super) struct HashJoinStream { /// Optional future to signal when build information has been reported by all partitions /// and the dynamic filter has been updated build_waiter: Option>, - /// Partitioning mode to use mode: PartitionMode, + /// Output buffer for coalescing small batches into larger ones. + /// Uses `BatchCoalescer` from arrow to efficiently combine batches. + /// When batches are already close to target size, they bypass coalescing. + output_buffer: Box, } impl RecordBatchStream for HashJoinStream { @@ -349,6 +353,14 @@ impl HashJoinStream { build_accumulator: Option>, mode: PartitionMode, ) -> Self { + // Create output buffer with coalescing. + // Use biggest_coalesce_batch_size to bypass coalescing for batches + // that are already close to target size (within 50%). + let output_buffer = Box::new( + BatchCoalescer::new(Arc::clone(&schema), batch_size) + .with_biggest_coalesce_batch_size(Some(batch_size / 2)), + ); + Self { partition, schema, @@ -368,6 +380,7 @@ impl HashJoinStream { build_accumulator, build_waiter: None, mode, + output_buffer, } } @@ -378,6 +391,11 @@ impl HashJoinStream { cx: &mut std::task::Context<'_>, ) -> Poll>> { loop { + // First, check if we have any completed batches ready to emit + if let Some(batch) = self.output_buffer.next_completed_batch() { + return self.join_metrics.baseline.record_poll(Poll::Ready(Some(Ok(batch)))); + } + return match self.state { HashJoinStreamState::WaitBuildSide => { handle_state!(ready!(self.collect_build_side(cx))) @@ -389,14 +407,22 @@ impl HashJoinStream { handle_state!(ready!(self.fetch_probe_batch(cx))) } HashJoinStreamState::ProcessProbeBatch(_) => { - let poll = handle_state!(self.process_probe_batch()); - self.join_metrics.baseline.record_poll(poll) + handle_state!(self.process_probe_batch()) } HashJoinStreamState::ExhaustedProbeSide => { - let poll = handle_state!(self.process_unmatched_build_batch()); - self.join_metrics.baseline.record_poll(poll) + handle_state!(self.process_unmatched_build_batch()) + } + HashJoinStreamState::Completed => { + // Flush any remaining buffered data + if !self.output_buffer.is_empty() { + self.output_buffer + .finish_buffered_batch() + .map_err(|e| -> datafusion_common::DataFusionError { e.into() })?; + // Continue loop to emit the flushed batch + continue; + } + Poll::Ready(None) } - HashJoinStreamState::Completed => Poll::Ready(None), }; } } @@ -628,28 +654,25 @@ impl HashJoinStream { self.right_side_ordered, )?; - let result = if self.join_type == JoinType::RightMark { - build_batch_from_indices( - &self.schema, - &state.batch, - build_side.left_data.batch(), - &left_indices, - &right_indices, - &self.column_indices, - JoinSide::Right, - )? + // Build output batch and push to coalescer + let (build_batch, probe_batch, join_side) = if self.join_type == JoinType::RightMark { + (&state.batch, build_side.left_data.batch(), JoinSide::Right) } else { - build_batch_from_indices( - &self.schema, - build_side.left_data.batch(), - &state.batch, - &left_indices, - &right_indices, - &self.column_indices, - JoinSide::Left, - )? + (build_side.left_data.batch(), &state.batch, JoinSide::Left) }; + let batch = build_batch_from_indices( + &self.schema, + build_batch, + probe_batch, + &left_indices, + &right_indices, + &self.column_indices, + join_side, + )?; + + self.output_buffer.push_batch(batch)?; + timer.done(); if next_offset.is_none() { @@ -662,7 +685,7 @@ impl HashJoinStream { ) }; - Ok(StatefulStreamResult::Ready(Some(result))) + Ok(StatefulStreamResult::Continue) } /// Processes unmatched build-side rows for certain join types and produces output batch @@ -690,27 +713,30 @@ impl HashJoinStream { self.join_type, true, ); - let empty_right_batch = RecordBatch::new_empty(self.right.schema()); - // use the left and right indices to produce the batch result - let result = build_batch_from_indices( - &self.schema, - build_side.left_data.batch(), - &empty_right_batch, - &left_side, - &right_side, - &self.column_indices, - JoinSide::Left, - ); - if let Ok(ref batch) = result { - self.join_metrics.input_batches.add(1); - self.join_metrics.input_rows.add(batch.num_rows()); - } + self.join_metrics.input_batches.add(1); + self.join_metrics.input_rows.add(left_side.len()); + timer.done(); self.state = HashJoinStreamState::Completed; - Ok(StatefulStreamResult::Ready(Some(result?))) + // Push final unmatched indices to output buffer + if !left_side.is_empty() { + let empty_right_batch = RecordBatch::new_empty(self.right.schema()); + let batch = build_batch_from_indices( + &self.schema, + build_side.left_data.batch(), + &empty_right_batch, + &left_side, + &right_side, + &self.column_indices, + JoinSide::Left, + )?; + self.output_buffer.push_batch(batch)?; + } + + Ok(StatefulStreamResult::Continue) } } diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 03900a608e6a..157adc237354 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -905,10 +905,9 @@ logical_plan 04)----SubqueryAlias: cte 05)------TableScan: person projection=[id] physical_plan -01)CoalesceBatchesExec: target_batch_size=8182 -02)--HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index e5cd6d88b08f..e4cb48f8353d 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -124,10 +124,9 @@ logical_plan 06)------TableScan: right_parquet projection=[id, info] physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Disable Join dynamic filter pushdown statement ok @@ -148,10 +147,9 @@ logical_plan 06)------TableScan: right_parquet projection=[id, info] physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Re-enable for next tests statement ok @@ -181,10 +179,9 @@ logical_plan 06)------TableScan: right_parquet projection=[id, info] physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Enable TopK, disable Join statement ok @@ -208,10 +205,9 @@ logical_plan 06)------TableScan: right_parquet projection=[id, info] physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Test 4: Backward compatibility @@ -260,10 +256,9 @@ logical_plan 06)------TableScan: right_parquet projection=[id, info] physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Re-enable statement ok @@ -303,10 +298,9 @@ logical_plan 06)------TableScan: right_parquet projection=[id, info] physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Cleanup diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 6ac28997a990..0fa7c3b791d4 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -298,36 +298,30 @@ physical_plan 07)│ string_col │ 08)└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐ -10)│ CoalesceBatchesExec │ +10)│ HashJoinExec │ 11)│ -------------------- │ -12)│ target_batch_size: │ -13)│ 8192 │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ HashJoinExec │ -17)│ -------------------- │ -18)│ filter: │ -19)│ CAST(int_col + int_col AS │ -20)│ Int64) % 2 = 0 ├──────────────┐ -21)│ │ │ -22)│ on: │ │ -23)│ (int_col = int_col) │ │ -24)└─────────────┬─────────────┘ │ -25)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -26)│ DataSourceExec ││ RepartitionExec │ -27)│ -------------------- ││ -------------------- │ -28)│ files: 1 ││ partition_count(in->out): │ -29)│ format: parquet ││ 1 -> 4 │ -30)│ ││ │ -31)│ ││ partitioning_scheme: │ -32)│ ││ RoundRobinBatch(4) │ -33)└───────────────────────────┘└─────────────┬─────────────┘ -34)-----------------------------┌─────────────┴─────────────┐ -35)-----------------------------│ DataSourceExec │ -36)-----------------------------│ -------------------- │ -37)-----------------------------│ files: 1 │ -38)-----------------------------│ format: csv │ -39)-----------------------------└───────────────────────────┘ +12)│ filter: │ +13)│ CAST(int_col + int_col AS │ +14)│ Int64) % 2 = 0 ├──────────────┐ +15)│ │ │ +16)│ on: │ │ +17)│ (int_col = int_col) │ │ +18)└─────────────┬─────────────┘ │ +19)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +20)│ DataSourceExec ││ RepartitionExec │ +21)│ -------------------- ││ -------------------- │ +22)│ files: 1 ││ partition_count(in->out): │ +23)│ format: parquet ││ 1 -> 4 │ +24)│ ││ │ +25)│ ││ partitioning_scheme: │ +26)│ ││ RoundRobinBatch(4) │ +27)└───────────────────────────┘└─────────────┬─────────────┘ +28)-----------------------------┌─────────────┴─────────────┐ +29)-----------------------------│ DataSourceExec │ +30)-----------------------------│ -------------------- │ +31)-----------------------------│ files: 1 │ +32)-----------------------------│ format: csv │ +33)-----------------------------└───────────────────────────┘ # 3 Joins query TT @@ -349,53 +343,41 @@ physical_plan 07)│ string_col │ 08)└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐ -10)│ CoalesceBatchesExec │ +10)│ HashJoinExec │ 11)│ -------------------- │ -12)│ target_batch_size: │ -13)│ 8192 │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ HashJoinExec │ -17)│ -------------------- │ -18)│ on: ├──────────────┐ -19)│ (int_col = int_col) │ │ -20)└─────────────┬─────────────┘ │ -21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ DataSourceExec ││ ProjectionExec │ -23)│ -------------------- ││ -------------------- │ -24)│ bytes: 520 ││ date_col: date_col │ -25)│ format: memory ││ int_col: int_col │ -26)│ rows: 1 ││ │ -27)│ ││ string_col: │ -28)│ ││ string_col │ -29)└───────────────────────────┘└─────────────┬─────────────┘ -30)-----------------------------┌─────────────┴─────────────┐ -31)-----------------------------│ CoalesceBatchesExec │ -32)-----------------------------│ -------------------- │ -33)-----------------------------│ target_batch_size: │ -34)-----------------------------│ 8192 │ -35)-----------------------------└─────────────┬─────────────┘ -36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ HashJoinExec │ -38)-----------------------------│ -------------------- │ -39)-----------------------------│ on: ├──────────────┐ -40)-----------------------------│ (int_col = int_col) │ │ -41)-----------------------------└─────────────┬─────────────┘ │ -42)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -43)-----------------------------│ DataSourceExec ││ RepartitionExec │ -44)-----------------------------│ -------------------- ││ -------------------- │ -45)-----------------------------│ files: 1 ││ partition_count(in->out): │ -46)-----------------------------│ format: parquet ││ 1 -> 4 │ -47)-----------------------------│ ││ │ -48)-----------------------------│ predicate: ││ partitioning_scheme: │ -49)-----------------------------│ DynamicFilter [ empty ] ││ RoundRobinBatch(4) │ -50)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ -51)----------------------------------------------------------┌─────────────┴─────────────┐ -52)----------------------------------------------------------│ DataSourceExec │ -53)----------------------------------------------------------│ -------------------- │ -54)----------------------------------------------------------│ files: 1 │ -55)----------------------------------------------------------│ format: csv │ -56)----------------------------------------------------------└───────────────────────────┘ +12)│ on: ├──────────────┐ +13)│ (int_col = int_col) │ │ +14)└─────────────┬─────────────┘ │ +15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +16)│ DataSourceExec ││ ProjectionExec │ +17)│ -------------------- ││ -------------------- │ +18)│ bytes: 520 ││ date_col: date_col │ +19)│ format: memory ││ int_col: int_col │ +20)│ rows: 1 ││ │ +21)│ ││ string_col: │ +22)│ ││ string_col │ +23)└───────────────────────────┘└─────────────┬─────────────┘ +24)-----------------------------┌─────────────┴─────────────┐ +25)-----------------------------│ HashJoinExec │ +26)-----------------------------│ -------------------- │ +27)-----------------------------│ on: ├──────────────┐ +28)-----------------------------│ (int_col = int_col) │ │ +29)-----------------------------└─────────────┬─────────────┘ │ +30)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +31)-----------------------------│ DataSourceExec ││ RepartitionExec │ +32)-----------------------------│ -------------------- ││ -------------------- │ +33)-----------------------------│ files: 1 ││ partition_count(in->out): │ +34)-----------------------------│ format: parquet ││ 1 -> 4 │ +35)-----------------------------│ ││ │ +36)-----------------------------│ predicate: ││ partitioning_scheme: │ +37)-----------------------------│ DynamicFilter [ empty ] ││ RoundRobinBatch(4) │ +38)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ +39)----------------------------------------------------------┌─────────────┴─────────────┐ +40)----------------------------------------------------------│ DataSourceExec │ +41)----------------------------------------------------------│ -------------------- │ +42)----------------------------------------------------------│ files: 1 │ +43)----------------------------------------------------------│ format: csv │ +44)----------------------------------------------------------└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -1123,34 +1105,28 @@ physical_plan 11)│ string_col │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ CoalesceBatchesExec │ +14)│ HashJoinExec │ 15)│ -------------------- │ -16)│ target_batch_size: │ -17)│ 8192 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ HashJoinExec │ -21)│ -------------------- │ -22)│ on: │ -23)│ (int_col = int_col), ├──────────────┐ -24)│ (string_col = │ │ -25)│ string_col) │ │ -26)└─────────────┬─────────────┘ │ -27)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -28)│ DataSourceExec ││ RepartitionExec │ -29)│ -------------------- ││ -------------------- │ -30)│ files: 1 ││ partition_count(in->out): │ -31)│ format: parquet ││ 1 -> 4 │ -32)│ ││ │ -33)│ ││ partitioning_scheme: │ -34)│ ││ RoundRobinBatch(4) │ -35)└───────────────────────────┘└─────────────┬─────────────┘ -36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ DataSourceExec │ -38)-----------------------------│ -------------------- │ -39)-----------------------------│ files: 1 │ -40)-----------------------------│ format: csv │ -41)-----------------------------└───────────────────────────┘ +16)│ on: │ +17)│ (int_col = int_col), ├──────────────┐ +18)│ (string_col = │ │ +19)│ string_col) │ │ +20)└─────────────┬─────────────┘ │ +21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +22)│ DataSourceExec ││ RepartitionExec │ +23)│ -------------------- ││ -------------------- │ +24)│ files: 1 ││ partition_count(in->out): │ +25)│ format: parquet ││ 1 -> 4 │ +26)│ ││ │ +27)│ ││ partitioning_scheme: │ +28)│ ││ RoundRobinBatch(4) │ +29)└───────────────────────────┘└─────────────┬─────────────┘ +30)-----------------------------┌─────────────┴─────────────┐ +31)-----------------------------│ DataSourceExec │ +32)-----------------------------│ -------------------- │ +33)-----------------------------│ files: 1 │ +34)-----------------------------│ format: csv │ +35)-----------------------------└───────────────────────────┘ # Query with outer hash join. query TT @@ -1170,36 +1146,30 @@ physical_plan 11)│ string_col │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ -14)│ CoalesceBatchesExec │ +14)│ HashJoinExec │ 15)│ -------------------- │ -16)│ target_batch_size: │ -17)│ 8192 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ HashJoinExec │ -21)│ -------------------- │ -22)│ join_type: Right │ -23)│ │ -24)│ on: ├──────────────┐ -25)│ (int_col = int_col), │ │ -26)│ (string_col = │ │ -27)│ string_col) │ │ -28)└─────────────┬─────────────┘ │ -29)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -30)│ DataSourceExec ││ RepartitionExec │ -31)│ -------------------- ││ -------------------- │ -32)│ files: 1 ││ partition_count(in->out): │ -33)│ format: parquet ││ 1 -> 4 │ -34)│ ││ │ -35)│ ││ partitioning_scheme: │ -36)│ ││ RoundRobinBatch(4) │ -37)└───────────────────────────┘└─────────────┬─────────────┘ -38)-----------------------------┌─────────────┴─────────────┐ -39)-----------------------------│ DataSourceExec │ -40)-----------------------------│ -------------------- │ -41)-----------------------------│ files: 1 │ -42)-----------------------------│ format: csv │ -43)-----------------------------└───────────────────────────┘ +16)│ join_type: Right │ +17)│ │ +18)│ on: ├──────────────┐ +19)│ (int_col = int_col), │ │ +20)│ (string_col = │ │ +21)│ string_col) │ │ +22)└─────────────┬─────────────┘ │ +23)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +24)│ DataSourceExec ││ RepartitionExec │ +25)│ -------------------- ││ -------------------- │ +26)│ files: 1 ││ partition_count(in->out): │ +27)│ format: parquet ││ 1 -> 4 │ +28)│ ││ │ +29)│ ││ partitioning_scheme: │ +30)│ ││ RoundRobinBatch(4) │ +31)└───────────────────────────┘└─────────────┬─────────────┘ +32)-----------------------------┌─────────────┴─────────────┐ +33)-----------------------------│ DataSourceExec │ +34)-----------------------------│ -------------------- │ +35)-----------------------------│ files: 1 │ +36)-----------------------------│ format: csv │ +37)-----------------------------└───────────────────────────┘ # Query with nested loop join. query TT diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index b74815edaa57..cecaa1a7deff 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2022,10 +2022,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] -12)----------------------DataSourceExec: partitions=1, partition_sizes=[3] -13)----------------------DataSourceExec: partitions=1, partition_sizes=[3] +10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[3] +12)--------------------DataSourceExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2944,10 +2943,9 @@ physical_plan 02)--ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, last_value(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] 03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[last_value(e.amount) ORDER BY [e.sn ASC NULLS LAST]] 04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -3937,12 +3935,11 @@ logical_plan physical_plan 01)ProjectionExec: expr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true -06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +06)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # reset partition number to 8. statement ok @@ -4093,14 +4090,13 @@ logical_plan 10)----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan 01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] -04)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true -07)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -08)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -09)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] +03)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +04)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +06)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +07)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -5167,10 +5163,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))@0 as ts_chunk, count(keywords_stream.keyword)@1 as alert_keyword_count] 02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] -05)--------DataSourceExec: partitions=1, partition_sizes=[3] -06)--------DataSourceExec: partitions=1, partition_sizes=[3] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] +04)------DataSourceExec: partitions=1, partition_sizes=[3] +05)------DataSourceExec: partitions=1, partition_sizes=[3] query PI SELECT diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index ea998f742d30..cb405065f168 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -776,10 +776,9 @@ logical_plan 03)--SubqueryAlias: t2 04)----TableScan: t1 projection=[a, b] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] # Reset the configs to old values statement ok @@ -985,11 +984,10 @@ logical_plan physical_plan 01)FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] -05)--------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] +04)------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1177,16 +1175,14 @@ logical_plan 06)--------TableScan: t5 projection=[v0, v1, v2, v3, v4] 07)----TableScan: t0 projection=[v0, v1] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(t1.v0 AS Float64)@6, v1@1)], filter=v1@1 + CAST(v0@0 AS Float64) > 0, projection=[v0@0, v1@1, v2@3, v3@4, v4@5, v0@7, v1@8] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[v0@0 as v0, v1@1 as v1, v0@2 as v0, v2@3 as v2, v3@4 as v3, v4@5 as v4, CAST(v0@0 AS Float64) as CAST(t1.v0 AS Float64)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] -08)--------------DataSourceExec: partitions=1, partition_sizes=[0] -09)--------------DataSourceExec: partitions=1, partition_sizes=[0] -10)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(t1.v0 AS Float64)@6, v1@1)], filter=v1@1 + CAST(v0@0 AS Float64) > 0, projection=[v0@0, v1@1, v2@3, v3@4, v4@5, v0@7, v1@8] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[v0@0 as v0, v1@1 as v1, v0@2 as v0, v2@3 as v2, v3@4 as v3, v4@5 as v4, CAST(v0@0 AS Float64) as CAST(t1.v0 AS Float64)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] +06)----------DataSourceExec: partitions=1, partition_sizes=[0] +07)----------DataSourceExec: partitions=1, partition_sizes=[0] +08)--DataSourceExec: partitions=1, partition_sizes=[0] @@ -1372,19 +1368,17 @@ logical_plan 07)--TableScan: s projection=[b] physical_plan 01)ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@3 as a, b@0 as b] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] -04)------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] -07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] -09)------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] -10)--------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] -12)--------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -13)----------------FilterExec: y@1 = x@0 -14)------------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] +03)----ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +06)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +07)--------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +08)----------DataSourceExec: partitions=1, partition_sizes=[1] +09)--------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +10)----------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +11)------------FilterExec: y@1 = x@0 +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; @@ -1428,16 +1422,15 @@ logical_plan 06)--------TableScan: t1 projection=[v0, v1] physical_plan 01)ProjectionExec: expr=[v0@1 as v0, v1@2 as v1, sum(t1.v1)@0 as sum(t1.v1)] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[sum(t1.v1)@0, v0@2, v1@3] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] -06)----------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[sum(t1.v1)@0, v0@2, v1@3] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] +05)--------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * @@ -1457,10 +1450,9 @@ logical_plan 02)--TableScan: t0 projection=[v0, v1] 03)--TableScan: t1 projection=[v0, v1] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] query IIII SELECT * FROM t0, LATERAL (SELECT * FROM t1 WHERE t0.v0 = t1.v0); diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index e78169734fe5..67a1d35ed61a 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -55,11 +55,10 @@ logical_plan 07)--------TableScan: annotated_data projection=[a, c] physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 -02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true -05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true +04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # preserve_inner_join query IIII nosort @@ -96,12 +95,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] -03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -06)--------FilterExec: d@3 = 3 -07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true +05)------FilterExec: d@3 = 3 +06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt index 166d1a028ecb..8246f489c446 100644 --- a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt +++ b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt @@ -80,10 +80,9 @@ logical_plan 04)----TableScan: t2 projection=[id, val] physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.format = "tree"; @@ -103,25 +102,19 @@ physical_plan 06)│ val: val │ 07)└─────────────┬─────────────┘ 08)┌─────────────┴─────────────┐ -09)│ CoalesceBatchesExec │ +09)│ HashJoinExec │ 10)│ -------------------- │ -11)│ target_batch_size: │ -12)│ 8192 │ -13)└─────────────┬─────────────┘ -14)┌─────────────┴─────────────┐ -15)│ HashJoinExec │ -16)│ -------------------- │ -17)│ NullsEqual: true ├──────────────┐ -18)│ │ │ -19)│ on: (val = val) │ │ -20)└─────────────┬─────────────┘ │ -21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ DataSourceExec ││ DataSourceExec │ -23)│ -------------------- ││ -------------------- │ -24)│ bytes: 288 ││ bytes: 288 │ -25)│ format: memory ││ format: memory │ -26)│ rows: 1 ││ rows: 1 │ -27)└───────────────────────────┘└───────────────────────────┘ +11)│ NullsEqual: true ├──────────────┐ +12)│ │ │ +13)│ on: (val = val) │ │ +14)└─────────────┬─────────────┘ │ +15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +16)│ DataSourceExec ││ DataSourceExec │ +17)│ -------------------- ││ -------------------- │ +18)│ bytes: 288 ││ bytes: 288 │ +19)│ format: memory ││ format: memory │ +20)│ rows: 1 ││ rows: 1 │ +21)└───────────────────────────┘└───────────────────────────┘ statement ok set datafusion.explain.format = "indent"; @@ -148,15 +141,14 @@ logical_plan 05)----TableScan: t2 projection=[id, val] physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], projection=[id@0, val@1, id@3, val@4], NullsEqual: true -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] -10)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], projection=[id@0, val@1, id@3, val@4], NullsEqual: true +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------FilterExec: CAST(val@1 AS Int64) + 1 IS NOT DISTINCT FROM 11 +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] +09)------DataSourceExec: partitions=1, partition_sizes=[1] # Mixed join predicate with `IS DISTINCT FROM` and `IS NOT DISTINCT FROM` query IIII rowsort @@ -178,12 +170,11 @@ logical_plan 04)----TableScan: t2 projection=[id, val] physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], filter=CAST(val@0 AS Int64) % 3 IS DISTINCT FROM CAST(val@1 AS Int64) % 3, projection=[id@0, val@1, id@3, val@4], NullsEqual: true -04)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] -06)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] -07)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], filter=CAST(val@0 AS Int64) % 3 IS DISTINCT FROM CAST(val@1 AS Int64) % 3, projection=[id@0, val@1, id@3, val@4], NullsEqual: true +03)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] +06)------DataSourceExec: partitions=1, partition_sizes=[1] # Test mixed equal and IS NOT DISTINCT FROM conditions # The `IS NOT DISTINCT FROM` expr should NOT in HashJoin's `on` predicate @@ -199,10 +190,9 @@ logical_plan 04)----TableScan: t2 projection=[id, val] physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], filter=val@0 IS NOT DISTINCT FROM val@1 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], filter=val@0 IS NOT DISTINCT FROM val@1 +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] # Test the mixed condition join result query IIII rowsort @@ -240,14 +230,12 @@ logical_plan 06)----TableScan: t0 projection=[val] physical_plan 01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@0, val@1)], projection=[id@1, val@2, id@3, val@4], NullsEqual: true -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true -08)------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@0, val@1)], projection=[id@1, val@2, id@3, val@4], NullsEqual: true +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true +06)--------DataSourceExec: partitions=1, partition_sizes=[1] +07)--------DataSourceExec: partitions=1, partition_sizes=[1] # Test IS NOT DISTINCT FROM with multiple columns statement ok @@ -284,13 +272,12 @@ JOIN t4 ON (t3.val1 IS NOT DISTINCT FROM t4.val1) AND (t3.val2 IS NOT DISTINCT F ---- 01)Projection: t3.id AS t3_id, t4.id AS t4_id, t3.val1, t4.val1, t3.val2, t4.val2 01)ProjectionExec: expr=[id@0 as t3_id, id@3 as t4_id, val1@1 as val1, val1@4 as val1, val2@2 as val2, val2@5 as val2] -02)--CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val1@1, val1@1), (val2@2, val2@2)], NullsEqual: true 02)--Inner Join: t3.val1 = t4.val1, t3.val2 = t4.val2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val1@1, val1@1), (val2@2, val2@2)], NullsEqual: true +03)----DataSourceExec: partitions=1, partition_sizes=[1] 03)----TableScan: t3 projection=[id, val1, val2] -04)------DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----TableScan: t4 projection=[id, val1, val2] -05)------DataSourceExec: partitions=1, partition_sizes=[1] logical_plan physical_plan diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 4a243258a519..5c211b796d3e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1342,11 +1342,10 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------------DataSourceExec: partitions=1, partition_sizes=[1] +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)------------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1359,11 +1358,10 @@ logical_plan 02)--TableScan: join_t3 projection=[s3] 03)--TableScan: join_t4 projection=[s4] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)----DataSourceExec: partitions=1, partition_sizes=[1] query ?? select join_t3.s3, join_t4.s4 @@ -1400,11 +1398,10 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1429,11 +1426,10 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 08)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -09)----------------CoalesceBatchesExec: target_batch_size=2 -10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -12)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +11)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1492,15 +1488,14 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1518,15 +1513,14 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1546,15 +1540,14 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1572,15 +1565,14 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------DataSourceExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1601,12 +1593,11 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1625,12 +1616,11 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1651,14 +1641,13 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +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] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1677,14 +1666,13 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------DataSourceExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +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] # Select wildcard with expr key inner join @@ -1703,12 +1691,11 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1725,12 +1712,11 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2567,11 +2553,10 @@ logical_plan 04)--SubqueryAlias: t2 05)----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)----DataSourceExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2830,12 +2815,11 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2866,12 +2850,11 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -2923,12 +2906,11 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2959,12 +2941,11 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -3017,12 +2998,11 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3034,12 +3014,11 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3089,12 +3068,11 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3106,12 +3084,11 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3298,12 +3275,11 @@ logical_plan 07)--------WindowAggr: windowExpr=[[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +03)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +04)----BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # hash join should propagate ordering equivalence of the right side for RIGHT ANTI join. # Hence final requirement rn1 ASC is already satisfied at the end of HashJoinExec. @@ -3325,12 +3301,11 @@ logical_plan 07)--------WindowAggr: windowExpr=[[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true -04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] -06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true +03)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +04)----BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING": UInt64 }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # Test ordering preservation for RIGHT join query TT @@ -3348,10 +3323,9 @@ logical_plan 05)----SubqueryAlias: r_table 06)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@2, b@2)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@2, b@2)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3373,10 +3347,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a, last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] 02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true -06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3421,12 +3394,11 @@ logical_plan physical_plan 01)ProjectionExec: expr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true -06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] -08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true +05)------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +06)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] +07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # run query above in multiple partitions statement ok @@ -3460,14 +3432,13 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC 06)----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] -09)----------------CoalesceBatchesExec: target_batch_size=2 -10)------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true -12)----------------CoalesceBatchesExec: target_batch_size=2 -13)------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -14)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +11)--------------CoalesceBatchesExec: target_batch_size=2 +12)----------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +13)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT * @@ -3620,15 +3591,14 @@ logical_plan 10)------EmptyRelation: rows=1 physical_plan 01)ProjectionExec: expr=[c@2 as c, d@3 as d, e@0 as e, f@1 as f] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] -04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------PlaceholderRowExec -06)------UnionExec -07)--------ProjectionExec: expr=[1 as c, 2 as d] -08)----------PlaceholderRowExec -09)--------ProjectionExec: expr=[1 as c, 3 as d] -10)----------PlaceholderRowExec +02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] +03)----ProjectionExec: expr=[1 as e, 3 as f] +04)------PlaceholderRowExec +05)----UnionExec +06)------ProjectionExec: expr=[1 as c, 2 as d] +07)--------PlaceholderRowExec +08)------ProjectionExec: expr=[1 as c, 3 as d] +09)--------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3663,15 +3633,14 @@ logical_plan 10)------EmptyRelation: rows=1 physical_plan 01)ProjectionExec: expr=[c@2 as c, d@3 as d, e@0 as e, f@1 as f] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] -04)------ProjectionExec: expr=[1 as e, 3 as f] -05)--------PlaceholderRowExec -06)------UnionExec -07)--------ProjectionExec: expr=[1 as c, 2 as d] -08)----------PlaceholderRowExec -09)--------ProjectionExec: expr=[1 as c, 3 as d] -10)----------PlaceholderRowExec +02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] +03)----ProjectionExec: expr=[1 as e, 3 as f] +04)------PlaceholderRowExec +05)----UnionExec +06)------ProjectionExec: expr=[1 as c, 2 as d] +07)--------PlaceholderRowExec +08)------ProjectionExec: expr=[1 as c, 3 as d] +09)--------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3876,11 +3845,10 @@ logical_plan 06)------TableScan: right_table_no_nulls projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------DataSourceExec: partitions=1, partition_sizes=[2] -06)------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] +03)----SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +04)------DataSourceExec: partitions=1, partition_sizes=[2] +05)----DataSourceExec: partitions=1, partition_sizes=[2] @@ -3934,10 +3902,9 @@ logical_plan 05)----TableScan: right_table_no_nulls projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -04)------DataSourceExec: partitions=1, partition_sizes=[2] -05)------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] +03)----DataSourceExec: partitions=1, partition_sizes=[2] +04)----DataSourceExec: partitions=1, partition_sizes=[2] # Null build indices: @@ -3994,11 +3961,10 @@ logical_plan 06)------TableScan: right_table_no_nulls projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------DataSourceExec: partitions=1, partition_sizes=[2] -06)------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] +03)----SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +04)------DataSourceExec: partitions=1, partition_sizes=[2] +05)----DataSourceExec: partitions=1, partition_sizes=[2] # Test CROSS JOIN LATERAL syntax (planning) @@ -4101,11 +4067,10 @@ physical_plan 01)SortExec: expr=[sn@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[ts@1 as ts, sn@0 as sn, amount@2 as amount, currency@3 as currency, CAST(amount@2 AS Float32) * last_value(e.rate)@4 as amount_usd] 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] -04)------CoalesceBatchesExec: target_batch_size=3 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] -06)----------DataSourceExec: partitions=1, partition_sizes=[0] -07)----------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -08)------------DataSourceExec: partitions=1, partition_sizes=[0] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] +05)--------DataSourceExec: partitions=1, partition_sizes=[0] +06)--------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +07)----------DataSourceExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4144,11 +4109,10 @@ logical_plan 03)----TableScan: left_table projection=[a, b, c] 04)----TableScan: right_table projection=[x, y, z] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, y@1)], filter=a@0 < x@1 -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] -05)------DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, y@1)], filter=a@0 < x@1 +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] +04)----DataSourceExec: partitions=1, partition_sizes=[0] # Test full join with limit statement ok @@ -4215,7 +4179,7 @@ logical_plan 03)----TableScan: t0 projection=[c1, c2] 04)----TableScan: t1 projection=[c1, c2, c3] physical_plan -01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[2] 04)----DataSourceExec: partitions=1, partition_sizes=[2] @@ -4245,7 +4209,7 @@ logical_plan 03)----TableScan: t0 projection=[c1, c2] 04)----TableScan: t1 projection=[c1, c2, c3] physical_plan -01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 03)----DataSourceExec: partitions=1, partition_sizes=[2] 04)----DataSourceExec: partitions=1, partition_sizes=[2] @@ -4309,7 +4273,7 @@ logical_plan 04)------TableScan: t1 projection=[a], fetch=2 05)----TableScan: t2 projection=[b] physical_plan -01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true @@ -4343,7 +4307,7 @@ logical_plan 04)----Limit: skip=0, fetch=2 05)------TableScan: t2 projection=[b], fetch=2 physical_plan -01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true @@ -4380,7 +4344,7 @@ logical_plan 03)----TableScan: t1 projection=[a] 04)----TableScan: t2 projection=[b] physical_plan -01)CoalesceBatchesExec: target_batch_size=3, fetch=2 +01)GlobalLimitExec: skip=0, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true @@ -4422,10 +4386,9 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)] 02)--AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1))] 03)----ProjectionExec: expr=[] -04)------CoalesceBatchesExec: target_batch_size=3 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------DataSourceExec: partitions=1, partition_sizes=[1] # Test hash join sort push down # Issue: https://github.com/apache/datafusion/issues/13559 @@ -4451,13 +4414,12 @@ logical_plan 07)----------TableScan: test projection=[a, b] physical_plan 01)SortPreservingMergeExec: [c@2 DESC] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] -04)------CoalescePartitionsExec -05)--------FilterExec: b@1 > 3, projection=[a@0] -06)----------DataSourceExec: partitions=2, partition_sizes=[1, 1] -07)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] +03)----CoalescePartitionsExec +04)------FilterExec: b@1 > 3, projection=[a@0] +05)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +06)----SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +07)------DataSourceExec: partitions=2, partition_sizes=[1, 1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4472,13 +4434,12 @@ logical_plan 07)----------TableScan: test projection=[a, b] physical_plan 01)SortPreservingMergeExec: [c@2 DESC NULLS LAST] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] -04)------CoalescePartitionsExec -05)--------FilterExec: b@1 > 3, projection=[a@0] -06)----------DataSourceExec: partitions=2, partition_sizes=[1, 1] -07)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] +03)----CoalescePartitionsExec +04)------FilterExec: b@1 > 3, projection=[a@0] +05)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] +06)----SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +07)------DataSourceExec: partitions=2, partition_sizes=[1, 1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; @@ -4516,10 +4477,9 @@ logical_plan 05)----SubqueryAlias: b 06)------TableScan: person projection=[id, age, state] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT age FROM (SELECT * FROM person a join person b using (id, age, state)); @@ -4532,10 +4492,9 @@ logical_plan 05)----SubqueryAlias: b 06)------TableScan: person projection=[id, age, state] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.* FROM person a join person b using (id, age); @@ -4548,10 +4507,9 @@ logical_plan 05)----SubqueryAlias: b 06)------TableScan: person projection=[id, age] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT a.*, b.* FROM person a join person b using (id, age); @@ -4563,10 +4521,9 @@ logical_plan 04)--SubqueryAlias: b 05)----TableScan: person projection=[id, age, state] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a join person b using (id, age, state) join person c using (id, age, state); @@ -4583,13 +4540,11 @@ logical_plan 09)----SubqueryAlias: c 10)------TableScan: person projection=[id, age, state] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] -03)----CoalesceBatchesExec: target_batch_size=3 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] -05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)--------DataSourceExec: partitions=1, partition_sizes=[0] -07)----DataSourceExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] +05)--DataSourceExec: partitions=1, partition_sizes=[0] query TT explain SELECT * FROM person a NATURAL JOIN lineitem b; @@ -4615,10 +4570,9 @@ logical_plan 04)----SubqueryAlias: lineitem2 05)------TableScan: lineitem projection=[c1] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -4751,12 +4705,11 @@ logical_plan 02)--TableScan: person projection=[id] 03)--TableScan: orders projection=[customer_id] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2] -03)----ProjectionExec: expr=[id@0 as id, CAST(id@0 AS Int64) + 10 as person.id + Int64(10)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] -06)------DataSourceExec: partitions=1, partition_sizes=[0] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2] +02)--ProjectionExec: expr=[id@0 as id, CAST(id@0 AS Int64) + 10 as person.id + Int64(10)] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)--ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] +05)----DataSourceExec: partitions=1, partition_sizes=[0] statement count 0 drop table person; @@ -4843,10 +4796,9 @@ logical_plan 04)----TableScan: t2 projection=[k] physical_plan 01)SortExec: TopK(fetch=2), expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------DataSourceExec: partitions=1, partition_sizes=[3334] +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[3334] query II @@ -4874,11 +4826,10 @@ logical_plan 03)----TableScan: t1 projection=[k, v] 04)----TableScan: t2 projection=[k] physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----SortExec: expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] -05)------DataSourceExec: partitions=1, partition_sizes=[3334] +01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--SortExec: expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] +04)----DataSourceExec: partitions=1, partition_sizes=[3334] statement ok DROP TABLE t1; @@ -4910,10 +4861,9 @@ JOIN t2 ON k1 = k2 ---- physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4931,10 +4881,9 @@ LEFT JOIN t2 ON k1 = k2 ---- physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4952,10 +4901,9 @@ RIGHT JOIN t2 ON k1 = k2 ---- physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4972,10 +4920,9 @@ FROM t1 LEFT SEMI JOIN t2 ON k1 = k2 ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(k2@0, k1@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k1), sum(v1) @@ -4992,10 +4939,9 @@ FROM t1 RIGHT SEMI JOIN t2 ON k1 = k2 ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(k2@0, k1@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k2), sum(v2) @@ -5012,10 +4958,9 @@ FROM t1 LEFT ANTI JOIN t2 ON k1 = k2 ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k1), sum(v1) @@ -5032,10 +4977,9 @@ FROM t1 RIGHT ANTI JOIN t2 ON k1 = k2 ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(k2@0, k1@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT sum(k2), sum(v2) @@ -5053,10 +4997,9 @@ FULL JOIN t2 ON k1 = k2 ---- physical_plan 01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Full, on=[(k2@0, k1@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(k2@0, k1@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -5079,10 +5022,9 @@ WHERE k2 > 0 ---- physical_plan 01)FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +02)--HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * @@ -5104,10 +5046,9 @@ LEFT ANTI JOIN t2 ON k1 = k2 ---- physical_plan 01)AggregateExec: mode=Single, gby=[v1@0 as v1], aggr=[] -02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], projection=[v1@1] -04)------DataSourceExec: partitions=1, partition_sizes=[0] -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], projection=[v1@1] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query I SELECT distinct(v1) @@ -5125,11 +5066,10 @@ LEFT ANTI JOIN t2 ON k1 = k2 WHERE k1 < 0 ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----FilterExec: k1@0 < 0 -05)------DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--FilterExec: k1@0 < 0 +04)----DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 769a42108b14..90c683947c17 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -668,18 +668,17 @@ logical_plan 05)----Filter: (part.p_brand = Utf8View("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) 06)------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_size <= Int32(15)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -05)--------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true -08)----CoalesceBatchesExec: target_batch_size=8192 -09)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -10)--------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 -11)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true +01)HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +04)------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true +07)--CoalesceBatchesExec: target_batch_size=8192 +08)----RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +09)------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -759,19 +758,17 @@ logical_plan physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[sum(lineitem.l_extendedprice), avg(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)] 02)--ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey, ps_suppkey@0 as ps_suppkey] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[ps_suppkey@1, l_extendedprice@2, l_discount@3, p_partkey@4] -05)--------DataSourceExec: partitions=1, partition_sizes=[1] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[ps_suppkey@1, l_extendedprice@2, l_discount@3, p_partkey@4] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] 06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -08)------------CoalesceBatchesExec: target_batch_size=8192 -09)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 -10)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true -11)------------CoalesceBatchesExec: target_batch_size=8192 -12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true +07)----------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 +08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true +09)--------CoalesceBatchesExec: target_batch_size=8192 +10)----------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +11)------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +12)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Simplification of a binary operator with a NULL value diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index cedc3f9f76a8..23f0ccbc91a8 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -299,11 +299,10 @@ query TT explain select * from small_table join large_table on small_table.k = large_table.k where large_table.v >= 50; ---- physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k@0, k@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/small_table.parquet]]}, projection=[k], file_type=parquet -04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k@0, k@0)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/small_table.parquet]]}, projection=[k], file_type=parquet +03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] statement ok drop table small_table; diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index aa87026c5cf3..5f9276bdb78e 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -939,4 +939,4 @@ SELECT t2.a, t2.b, t2.c FROM t2 WHERE t2.a > 3 OR t2.a IN (SELECT t3.x FROM t3 WHERE t2.b < 150) ---- -4 101 1001 \ No newline at end of file +4 101 1001 diff --git a/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt b/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt index e07c96d7c121..8ec886d02e78 100644 --- a/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt +++ b/datafusion/sqllogictest/test_files/spark/bitwise/bit_count.slt @@ -239,4 +239,4 @@ SELECT bit_count(false); query I SELECT bit_count(cast(null as boolean)); ---- -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/spark/math/csc.slt b/datafusion/sqllogictest/test_files/spark/math/csc.slt index 5eb9f4447280..837704113da4 100644 --- a/datafusion/sqllogictest/test_files/spark/math/csc.slt +++ b/datafusion/sqllogictest/test_files/spark/math/csc.slt @@ -43,4 +43,4 @@ SELECT csc(a) FROM (VALUES (pi()), (-pi()), (pi()/2) , (arrow_cast('NAN','Float3 8165619676597685 -8165619676597685 1 -NaN \ No newline at end of file +NaN diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 27325d4e5e84..b7c5a2bcbfe8 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -201,18 +201,17 @@ logical_plan 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: partitions=1, partition_sizes=[2] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------DataSourceExec: partitions=1, partition_sizes=[2] +11)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------DataSourceExec: partitions=1, partition_sizes=[2] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -236,18 +235,17 @@ logical_plan 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[sum(t2.t2_int * Float64(1))@1 + 1 as sum(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: partitions=1, partition_sizes=[2] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[sum(t2.t2_int * Float64(1))@1 + 1 as sum(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------DataSourceExec: partitions=1, partition_sizes=[2] +11)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------DataSourceExec: partitions=1, partition_sizes=[2] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -271,18 +269,17 @@ logical_plan 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------DataSourceExec: partitions=1, partition_sizes=[2] -12)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------DataSourceExec: partitions=1, partition_sizes=[2] +11)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)------DataSourceExec: partitions=1, partition_sizes=[2] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id group by t2_id, 'a') as t2_sum from t1 @@ -307,19 +304,18 @@ logical_plan 08)------------TableScan: t2 projection=[t2_id, t2_int] physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] -04)------CoalescePartitionsExec -05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------FilterExec: sum(t2.t2_int)@1 < 3 -07)------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -08)--------------CoalesceBatchesExec: target_batch_size=2 -09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------DataSourceExec: partitions=1, partition_sizes=[2] -13)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +03)----CoalescePartitionsExec +04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] +05)--------FilterExec: sum(t2.t2_int)@1 < 3 +06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------DataSourceExec: partitions=1, partition_sizes=[2] +12)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------DataSourceExec: partitions=1, partition_sizes=[2] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 @@ -1189,11 +1185,10 @@ logical_plan 06)--------TableScan: t2 projection=[t2_id] physical_plan 01)FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(t2_id@0, t1_id@0)] -04)------DataSourceExec: partitions=1, partition_sizes=[2] -05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[2] +02)--HashJoinExec: mode=CollectLeft, join_type=RightMark, on=[(t2_id@0, t1_id@0)] +03)----DataSourceExec: partitions=1, partition_sizes=[2] +04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[2] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index f848a330b276..7e087da5af3c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -76,27 +76,24 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------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 07)------------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)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------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] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +08)--------------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] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +11)--------------------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] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------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] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------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] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -20)--------------------------------------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 -21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -23)--------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -24)----------------------------------------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 -25)--------------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -27)------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -28)--------------------------------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 -29)------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 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 +13)------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +14)--------------------------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] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +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_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=csv, has_header=false +18)----------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +20)--------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +21)----------------------------------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 +22)----------------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +24)--------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +25)----------------------------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 +26)----------------CoalesceBatchesExec: target_batch_size=8192 +27)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +28)--------------------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 6d3831cfaeaf..cb2271ca651a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -82,41 +82,37 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +13)------------------------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] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------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] -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -18)----------------------------------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 -19)------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -21)----------------------------------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 -22)----------------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -25)----------------------------RepartitionExec: partitioning=RoundRobinBatch(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], file_type=csv, has_header=false -27)------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)] -28)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -29)----------CoalescePartitionsExec -30)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -31)--------------CoalesceBatchesExec: target_batch_size=8192 -32)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -33)------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -35)----------------------CoalesceBatchesExec: target_batch_size=8192 -36)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -37)--------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 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_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -40)--------------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -42)------------------------------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 -43)------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -45)----------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)--------------------------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 +15)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +16)------------------------------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 +17)--------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +19)------------------------------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 +20)--------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +22)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +23)--------------------------RepartitionExec: partitioning=RoundRobinBatch(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], file_type=csv, has_header=false +25)------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)] +26)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +27)----------CoalescePartitionsExec +28)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +29)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +30)----------------CoalesceBatchesExec: target_batch_size=8192 +31)------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +32)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +33)----------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +35)--------------------------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 +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +38)--------------------------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 +39)----------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +41)--------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +42)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +43)------------------------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 c2bc4d8511ef..f1008fead1bb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -66,12 +66,11 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 07)------------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)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------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] -13)------------------------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 -14)------------------CoalesceBatchesExec: target_batch_size=8192 -15)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -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_orderpriority], file_type=csv, has_header=false +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +11)--------------------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] +12)----------------------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 +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +15)--------------------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 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 65531be833fd..f05bb6648a93 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -62,12 +62,11 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] 08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false -15)----------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -17)--------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, 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_comment], file_type=csv, has_header=false +10)------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +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], file_type=csv, has_header=false +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +16)------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +17)--------------------------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_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 32e05fa583d6..b51c142406f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -46,12 +46,11 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------ProjectionExec: expr=[l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as __common_expr_1, p_type@2 as p_type] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] -11)--------------------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_partkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -12)--------------CoalesceBatchesExec: target_batch_size=8192 -13)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 -14)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +09)----------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] +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_partkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +11)------------CoalesceBatchesExec: target_batch_size=8192 +12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +13)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index d7275347bd1e..9f1534aa1bb9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -73,27 +73,25 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] 02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(max(revenue0.total_revenue)@0, total_revenue@4)], projection=[s_suppkey@1, s_name@2, s_address@3, s_phone@4, total_revenue@5] -05)--------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] -06)----------CoalescePartitionsExec -07)------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] -08)--------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -12)----------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -13)------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -14)--------------------------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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(max(revenue0.total_revenue)@0, total_revenue@4)], projection=[s_suppkey@1, s_name@2, s_address@3, s_phone@4, total_revenue@5] +04)------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] +05)--------CoalescePartitionsExec +06)----------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] +07)------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +11)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +12)----------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +13)------------------------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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +14)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] 15)--------CoalesceBatchesExec: target_batch_size=8192 -16)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] -17)------------CoalesceBatchesExec: target_batch_size=8192 -18)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -19)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false -20)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -21)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -22)----------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -24)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -25)----------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -26)------------------------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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +16)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +17)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false +18)--------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +19)----------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +20)------------CoalesceBatchesExec: target_batch_size=8192 +21)--------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +22)----------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +23)------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +24)--------------------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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index fca427dd8632..0537f7e5f3b1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -76,22 +76,20 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +12)----------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -20)--------------------------------------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], file_type=csv, has_header=false -21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -23)--------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]) -24)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -25)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false -26)--------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -28)------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -29)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false +17)--------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +18)----------------------------------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], file_type=csv, has_header=false +19)------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +21)----------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND p_size@3 IN (SET) ([49, 14, 23, 45, 19, 3, 36, 9]) +22)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false +24)------------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +26)----------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index e2123cfc7e56..433082c5546e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -55,21 +55,19 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -11)--------------------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_partkey, l_quantity, l_extendedprice], file_type=csv, has_header=false -12)----------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] -15)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false -17)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] -18)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -19)----------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -21)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -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_partkey, l_quantity], file_type=csv, has_header=false +08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=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_partkey, l_quantity, l_extendedprice], file_type=csv, has_header=false +10)------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +12)----------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +13)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false +15)----------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] +16)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +17)--------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +19)------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +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_partkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 9225f21a61e5..39f06a8f454c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -73,26 +73,23 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] +07)------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -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_name], file_type=csv, has_header=false -18)----------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -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_totalprice, o_orderdate], file_type=csv, has_header=false -21)--------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -23)------------------------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_quantity], file_type=csv, has_header=false -24)----------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -25)------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -26)--------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -28)------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -29)--------------------------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_quantity], file_type=csv, has_header=false +10)------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 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, o_orderkey@2, o_totalprice@4, o_orderdate@5] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false +15)----------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +17)--------------------------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_totalprice, o_orderdate], file_type=csv, has_header=false +18)----------------CoalesceBatchesExec: target_batch_size=8192 +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_quantity], file_type=csv, has_header=false +21)--------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +22)----------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +23)------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +25)----------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +26)------------------------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_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index d20f090fa5b8..93b92942456b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -68,14 +68,13 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] -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_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false -11)------------CoalesceBatchesExec: target_batch_size=8192 -12)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -13)----------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 -14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +08)--------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] +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_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false +10)----------CoalesceBatchesExec: target_batch_size=8192 +11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +12)--------------FilterExec: (p_brand@1 = Brand#12 AND p_container@3 IN (SET) ([SM CASE, SM BOX, SM PACK, SM PKG]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN (SET) ([MED BAG, MED BOX, MED PKG, MED PACK]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN (SET) ([LG CASE, LG BOX, LG PACK, LG PKG]) AND p_size@2 <= 15) AND p_size@2 >= 1 +13)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], 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 bf412998b6f4..552837f8901b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -102,72 +102,64 @@ physical_plan 01)SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 -16)------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@3 as s_name, s_address@4 as s_address, s_nationkey@5 as s_nationkey, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost] -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] -19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -25)------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] -26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false -28)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -30)------------------------------------------------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 -31)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -33)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false -34)--------------------------CoalesceBatchesExec: target_batch_size=8192 -35)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -36)------------------------------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 -37)------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -39)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -40)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)--------------------------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 -42)----------CoalesceBatchesExec: target_batch_size=8192 -43)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -44)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -45)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -48)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -49)------------------------CoalesceBatchesExec: target_batch_size=8192 -50)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -51)----------------------------CoalesceBatchesExec: target_batch_size=8192 -52)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -53)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -55)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -57)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -59)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -61)------------------------------------------------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 -62)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -63)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -64)------------------------------------------------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 -65)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -66)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -67)----------------------------------------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 -68)----------------------------CoalesceBatchesExec: target_batch_size=8192 -69)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -70)--------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -71)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -72)------------------------------------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 +09)----------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 +13)------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@3 as s_name, s_address@4 as s_address, s_nationkey@5 as s_nationkey, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost] +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +20)--------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] +21)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false +23)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@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_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +26)----------------------------CoalesceBatchesExec: target_batch_size=8192 +27)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +28)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false +29)--------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 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, n_regionkey], file_type=csv, has_header=false +32)--------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +34)------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +35)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +36)----------------------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 +37)--------CoalesceBatchesExec: target_batch_size=8192 +38)----------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +39)------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +40)--------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +41)----------------CoalesceBatchesExec: target_batch_size=8192 +42)------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +43)--------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +44)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +45)------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +47)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +48)------------------------------CoalesceBatchesExec: target_batch_size=8192 +49)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +50)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +51)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +53)----------------------------------------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 +54)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +55)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +56)----------------------------------------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 +57)------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +59)----------------------------------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 +60)------------------------CoalesceBatchesExec: target_batch_size=8192 +61)--------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +62)----------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +63)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +64)--------------------------------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/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 9dc0f6822a1a..68ea7761732a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -83,40 +83,36 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +03)----HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=1 -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false -12)----------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -14)--------------------FilterExec: n_name@1 = CANADA, 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)--------CoalesceBatchesExec: target_batch_size=8192 -18)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -19)------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -21)----------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -23)--------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -25)------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -27)----------------------------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], file_type=csv, has_header=false -28)------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -30)----------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -31)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -33)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -34)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -35)--------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -37)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -38)--------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -39)----------------------------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_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false +08)--------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=1 +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false +10)------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +12)----------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +13)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------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 +15)------CoalesceBatchesExec: target_batch_size=8192 +16)--------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +17)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +18)------------CoalesceBatchesExec: target_batch_size=8192 +19)--------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +20)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +21)------------------CoalesceBatchesExec: target_batch_size=8192 +22)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +23)----------------------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], file_type=csv, has_header=false +24)------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +26)----------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +27)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +29)------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +30)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +31)----------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +33)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +34)----------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +35)------------------------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_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index adb1b9bd10cc..254fbc7991d9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -97,42 +97,37 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +08)--------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -26)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false -27)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -29)--------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -30)----------------------------------------------------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_commitdate, l_receiptdate], file_type=csv, has_header=false -31)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -33)------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -34)--------------------------------------------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_orderstatus], file_type=csv, has_header=false -35)------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -37)----------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -38)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------------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 -40)----------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -42)--------------------------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], file_type=csv, has_header=false -43)------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -45)----------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -46)------------------------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_commitdate, l_receiptdate], file_type=csv, has_header=false +17)--------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +21)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false +22)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +24)----------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +25)------------------------------------------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_commitdate, l_receiptdate], file_type=csv, has_header=false +26)------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +28)----------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +29)------------------------------------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_orderstatus], file_type=csv, has_header=false +30)------------------------CoalesceBatchesExec: target_batch_size=8192 +31)--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +32)----------------------------FilterExec: n_name@1 = SAUDI ARABIA, 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 +35)------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +37)----------------------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], file_type=csv, has_header=false +38)----------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +40)--------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +41)----------------------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_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index a9d95fb1ab79..ef227430f31c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -86,19 +86,18 @@ physical_plan 10)------------------NestedLoopJoinExec: join_type=Inner, filter=join_proj_push_down_1@1 > avg(customer.c_acctbal)@0, projection=[c_phone@0, c_acctbal@1, avg(customer.c_acctbal)@3] 11)--------------------ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal, CAST(c_acctbal@1 AS Decimal128(19, 6)) as join_proj_push_down_1] 12)----------------------CoalescePartitionsExec -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -17)--------------------------------FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) -18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false -20)----------------------------CoalesceBatchesExec: target_batch_size=8192 -21)------------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -22)--------------------------------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_custkey], file_type=csv, has_header=false -23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -24)----------------------CoalescePartitionsExec -25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -26)--------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] -27)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false +13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +16)------------------------------FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) +17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false +19)--------------------------CoalesceBatchesExec: target_batch_size=8192 +20)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +21)------------------------------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_custkey], file_type=csv, has_header=false +22)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] +23)----------------------CoalescePartitionsExec +24)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] +25)--------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]), projection=[c_acctbal@1] +26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 27fd404644fa..225a18daffe3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -65,22 +65,20 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] -17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false -19)--------------------------CoalesceBatchesExec: target_batch_size=8192 -20)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -21)------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -22)--------------------------------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, o_shippriority], file_type=csv, has_header=false -23)------------------CoalesceBatchesExec: target_batch_size=8192 -24)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -25)----------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -26)------------------------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_shipdate], file_type=csv, has_header=false +13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +14)--------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false +17)----------------------CoalesceBatchesExec: target_batch_size=8192 +18)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +19)--------------------------FilterExec: o_orderdate@2 < 1995-03-15 +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, o_shippriority], file_type=csv, has_header=false +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +23)--------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +24)----------------------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_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 9f53448304a6..1adc37a91f11 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -60,13 +60,12 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -13)------------------------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_orderdate, o_orderpriority], file_type=csv, has_header=false -14)------------------CoalesceBatchesExec: target_batch_size=8192 -15)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -16)----------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -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_commitdate, l_receiptdate], file_type=csv, has_header=false +08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +11)--------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] +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_orderdate, o_orderpriority], file_type=csv, has_header=false +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +15)--------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +16)----------------------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_commitdate, l_receiptdate], 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 5c2fa1a40898..fdac01f51f7e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -74,42 +74,37 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +11)--------------------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] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------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] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------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] +13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +14)--------------------------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] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +17)--------------------------------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] 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------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] -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -28)------------------------------------------------------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 -29)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -31)------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -32)--------------------------------------------------------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 -33)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -35)----------------------------------------------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 -36)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 -38)--------------------------------------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 -39)--------------------------CoalesceBatchesExec: target_batch_size=8192 -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_name, n_regionkey], file_type=csv, has_header=false -42)------------------CoalesceBatchesExec: target_batch_size=8192 -43)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -44)----------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -45)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)--------------------------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 +19)------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +23)--------------------------------------------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 +24)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +26)--------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +27)----------------------------------------------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 +28)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +30)--------------------------------------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 +31)----------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 +33)--------------------------------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 +34)----------------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +36)--------------------------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 +37)----------------CoalesceBatchesExec: target_batch_size=8192 +38)------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +39)--------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +40)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)------------------------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/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index e19ae5c350e9..5abbe1ad5ab4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -92,44 +92,39 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] 08)--------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] +14)--------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 -21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -29)--------------------------------------------------------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 -30)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -32)--------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -33)----------------------------------------------------------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, l_shipdate], file_type=csv, has_header=false -34)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -36)------------------------------------------------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], file_type=csv, has_header=false -37)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -39)----------------------------------------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 -40)----------------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -42)--------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -43)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -44)------------------------------------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 -45)--------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -47)------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -48)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)----------------------------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)--------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 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)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +27)----------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +28)------------------------------------------------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, l_shipdate], file_type=csv, has_header=false +29)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +31)----------------------------------------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], file_type=csv, has_header=false +32)------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +34)----------------------------------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 +35)------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +37)----------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------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 +40)------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +42)----------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +43)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)--------------------------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/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 5d3ce5e56611..6281c2013e67 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -98,58 +98,51 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] 08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] +14)--------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 -21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +20)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +24)----------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] -31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -33)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] -35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] -38)--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)----------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false -40)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -42)------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -43)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -45)----------------------------------------------------------------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 -46)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -47)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -48)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -49)----------------------------------------------------------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 -50)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -51)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -52)------------------------------------------------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 -53)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -55)----------------------------------------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 -56)----------------------------CoalesceBatchesExec: target_batch_size=8192 -57)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -58)--------------------------------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 -59)--------------------CoalesceBatchesExec: target_batch_size=8192 -60)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -61)------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -62)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -63)----------------------------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 +26)--------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +27)----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] +28)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +30)----------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] +31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)--------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false +33)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +35)----------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +36)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)--------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +38)----------------------------------------------------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 +39)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +41)----------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +42)------------------------------------------------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 +43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +45)----------------------------------------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 +46)------------------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +48)----------------------------------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 +49)------------------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +51)----------------------------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 +52)------------------CoalesceBatchesExec: target_batch_size=8192 +53)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +54)----------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +55)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +56)--------------------------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/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index e4aa46dc15d1..88d52612e0b1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -83,41 +83,36 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 08)--------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] +14)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 -21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -29)--------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] -30)----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -31)------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false -32)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -34)--------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false -35)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -37)------------------------------------------------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 -38)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -40)----------------------------------------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 -41)----------------------------CoalesceBatchesExec: target_batch_size=8192 -42)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -43)--------------------------------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_orderdate], file_type=csv, has_header=false -44)--------------------CoalesceBatchesExec: target_batch_size=8192 -45)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -46)------------------------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)--------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +24)----------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] +25)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false +27)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)--------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +29)----------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false +30)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +32)----------------------------------------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 +33)------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +35)----------------------------------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 +36)------------------------CoalesceBatchesExec: target_batch_size=8192 +37)--------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +38)----------------------------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_orderdate], file_type=csv, has_header=false +39)------------------CoalesceBatchesExec: target_batch_size=8192 +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_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 9e63f79f4545..b714302b8c72 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -306,31 +306,29 @@ logical_plan 10)------TableScan: t1 projection=[id, name] physical_plan 01)UnionExec -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)], NullsEqual: true -04)------CoalescePartitionsExec -05)--------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -12)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)----------DataSourceExec: partitions=1, partition_sizes=[1] -14)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -15)----CoalesceBatchesExec: target_batch_size=2 -16)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true -17)--------CoalescePartitionsExec -18)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -19)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -20)--------------CoalesceBatchesExec: target_batch_size=2 -21)----------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -22)------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -23)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -24)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -25)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)----------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)], NullsEqual: true +03)----CoalescePartitionsExec +04)------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------DataSourceExec: partitions=1, partition_sizes=[1] +13)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true +15)------CoalescePartitionsExec +16)--------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +17)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +18)------------CoalesceBatchesExec: target_batch_size=2 +19)--------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +20)----------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +21)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +23)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +24)--------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -376,28 +374,26 @@ logical_plan 09)----TableScan: t1 projection=[name] physical_plan 01)UnionExec -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true -04)------CoalescePartitionsExec -05)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] -11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)--------DataSourceExec: partitions=1, partition_sizes=[1] -13)--CoalesceBatchesExec: target_batch_size=2 -14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true -15)------CoalescePartitionsExec -16)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -17)----------CoalesceBatchesExec: target_batch_size=2 -18)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -19)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -20)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)------------------DataSourceExec: partitions=1, partition_sizes=[1] -22)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -23)--------DataSourceExec: partitions=1, partition_sizes=[1] +02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true +03)----CoalescePartitionsExec +04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------DataSourceExec: partitions=1, partition_sizes=[1] +12)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true +13)----CoalescePartitionsExec +14)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +15)--------CoalesceBatchesExec: target_batch_size=2 +16)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +17)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +18)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)----------------DataSourceExec: partitions=1, partition_sizes=[1] +20)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)------DataSourceExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT From df11504c1836ecbf420ad696afa812ed951d09d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 27 Nov 2025 22:45:16 +0100 Subject: [PATCH 04/14] Move coalesce batches to hash join --- datafusion/core/tests/dataframe/mod.rs | 67 ++++++++++++------- datafusion/core/tests/sql/explain_analyze.rs | 14 ++-- .../src/coalesce_batches.rs | 16 ++--- .../src/joins/hash_join/stream.rs | 22 +++--- datafusion/sqllogictest/test_files/cte.slt | 15 ++--- .../sqllogictest/test_files/slt_features.slt | 2 +- datafusion/sqllogictest/test_files/union.slt | 20 +++--- 7 files changed, 87 insertions(+), 69 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 0c6ccf1b0751..d5367a5b665f 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2918,21 +2918,22 @@ async fn test_count_wildcard_on_sort() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+----------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+----------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) ASC NULLS LAST | - | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t1 projection=[b] | - | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | - | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | - | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+----------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __correlated_sq_1 | + | | Projection: count(Int64(1)) AS count(*) | + | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[] | + | physical_plan | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | + | | ProjectionExec: expr=[4 as count(*)] | + | | PlaceholderRowExec | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------------------------------------------------+ " ); Ok(()) @@ -3295,17 +3296,31 @@ async fn test_count_wildcard_on_aggregate() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), - @r###" - +---------------+---------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------+ - | logical_plan | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t1 projection=[] | - | physical_plan | ProjectionExec: expr=[4 as count(*)] | - | | PlaceholderRowExec | - | | | - +---------------+---------------------------------------------------------------+ - "### + @r" + +---------------+-----------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-----------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-----------------------------------------------------------------------------------------------------------------------+ + " ); Ok(()) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 1656bdf66f2c..7cc957b5d2e5 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -812,14 +812,14 @@ async fn test_physical_plan_display_indent_multi_children() { assert_snapshot!( actual, @r" - CoalesceBatchesExec: target_batch_size=4096 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0] - CoalesceBatchesExec: target_batch_size=4096 - RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=1 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0] + CoalesceBatchesExec: target_batch_size=4096 + RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=1 + DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true + CoalesceBatchesExec: target_batch_size=4096 + RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 + ProjectionExec: expr=[c1@0 as c2] DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true - CoalesceBatchesExec: target_batch_size=4096 - RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 - DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1@0 as c2], file_type=csv, has_header=true " ); } diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 02dce59cf49f..25042a09be9c 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -58,14 +58,14 @@ impl PhysicalOptimizerRule for CoalesceBatches { plan.transform_up(|plan| { let plan_any = plan.as_any(); let wrap_in_coalesce = plan_any - .downcast_ref::() - .map(|repart_exec| { - !matches!( - repart_exec.partitioning().clone(), - Partitioning::RoundRobinBatch(_) - ) - }) - .unwrap_or(false); + .downcast_ref::() + .map(|repart_exec| { + !matches!( + repart_exec.partitioning().clone(), + Partitioning::RoundRobinBatch(_) + ) + }) + .unwrap_or(false); if wrap_in_coalesce { Ok(Transformed::yes(Arc::new(CoalesceBatchesExec::new( diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index df6d262d110d..7d121b32c2b5 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -393,7 +393,10 @@ impl HashJoinStream { loop { // First, check if we have any completed batches ready to emit if let Some(batch) = self.output_buffer.next_completed_batch() { - return self.join_metrics.baseline.record_poll(Poll::Ready(Some(Ok(batch)))); + return self + .join_metrics + .baseline + .record_poll(Poll::Ready(Some(Ok(batch)))); } return match self.state { @@ -415,9 +418,9 @@ impl HashJoinStream { HashJoinStreamState::Completed => { // Flush any remaining buffered data if !self.output_buffer.is_empty() { - self.output_buffer - .finish_buffered_batch() - .map_err(|e| -> datafusion_common::DataFusionError { e.into() })?; + self.output_buffer.finish_buffered_batch().map_err( + |e| -> datafusion_common::DataFusionError { e.into() }, + )?; // Continue loop to emit the flushed batch continue; } @@ -655,11 +658,12 @@ impl HashJoinStream { )?; // Build output batch and push to coalescer - let (build_batch, probe_batch, join_side) = if self.join_type == JoinType::RightMark { - (&state.batch, build_side.left_data.batch(), JoinSide::Right) - } else { - (build_side.left_data.batch(), &state.batch, JoinSide::Left) - }; + let (build_batch, probe_batch, join_side) = + if self.join_type == JoinType::RightMark { + (&state.batch, build_side.left_data.batch(), JoinSide::Right) + } else { + (build_side.left_data.batch(), &state.batch, JoinSide::Left) + }; let batch = build_batch_from_indices( &self.schema, diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 41e90e4c3399..d6ef2dc4aae6 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -1098,14 +1098,13 @@ physical_plan 01)RecursiveQueryExec: name=trans, is_distinct=true 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/closure.csv]]}, projection=[start, end], file_type=csv, has_header=true 03)--CoalescePartitionsExec -04)----CoalesceBatchesExec: target_batch_size=8182 -05)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(end@1, start@0)], projection=[start@0, end@3] -06)--------CoalesceBatchesExec: target_batch_size=8182 -07)----------RepartitionExec: partitioning=Hash([end@1], 4), input_partitions=1 -08)------------WorkTableExec: name=trans -09)--------CoalesceBatchesExec: target_batch_size=8182 -10)----------RepartitionExec: partitioning=Hash([start@0], 4), input_partitions=1 -11)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/closure.csv]]}, projection=[start, end], file_type=csv, has_header=true +04)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(end@1, start@0)], projection=[start@0, end@3] +05)------CoalesceBatchesExec: target_batch_size=8182 +06)--------RepartitionExec: partitioning=Hash([end@1], 4), input_partitions=1 +07)----------WorkTableExec: name=trans +08)------CoalesceBatchesExec: target_batch_size=8182 +09)--------RepartitionExec: partitioning=Hash([start@0], 4), input_partitions=1 +10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/closure.csv]]}, projection=[start, end], file_type=csv, has_header=true statement count 0 set datafusion.execution.enable_recursive_ctes = false; diff --git a/datafusion/sqllogictest/test_files/slt_features.slt b/datafusion/sqllogictest/test_files/slt_features.slt index f3d467ea0d93..5075ed10eae9 100644 --- a/datafusion/sqllogictest/test_files/slt_features.slt +++ b/datafusion/sqllogictest/test_files/slt_features.slt @@ -71,4 +71,4 @@ select * from generate_series(3); 1 - \ No newline at end of file + diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 3b8599dd407c..d62360ae8c3b 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -306,17 +306,17 @@ logical_plan 10)------TableScan: t1 projection=[id, name] physical_plan 01)UnionExec -02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)], NullsEqual: true +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], NullsEqual: true 03)----CoalescePartitionsExec -04)------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)--------DataSourceExec: partitions=1, partition_sizes=[1] +04)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +07)----AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +10)----------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +11)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] 13)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true 15)------CoalescePartitionsExec From ac5f4082fdca8e47622a516adf0e3fc7f3e36c91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 27 Nov 2025 22:51:34 +0100 Subject: [PATCH 05/14] Fix tests --- datafusion/core/tests/dataframe/mod.rs | 149 ++++++++----------- datafusion/core/tests/sql/explain_analyze.rs | 3 +- 2 files changed, 67 insertions(+), 85 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index d5367a5b665f..5d4aee26d051 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2918,22 +2918,21 @@ async fn test_count_wildcard_on_sort() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+----------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+----------------------------------------------------------------------------------------------------------------------+ - | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | Projection: count(Int64(1)) AS count(*) | - | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[] | - | physical_plan | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | - | | ProjectionExec: expr=[4 as count(*)] | - | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+----------------------------------------------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) ASC NULLS LAST | + | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t1 projection=[b] | + | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | + | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | + | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------+ " ); Ok(()) @@ -2952,23 +2951,22 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | Projection: count(Int64(1)) AS count(*) | - | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | - | | ProjectionExec: expr=[4 as count(*)] | - | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __correlated_sq_1 | + | | Projection: count(Int64(1)) AS count(*) | + | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[] | + | physical_plan | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | + | | ProjectionExec: expr=[4 as count(*)] | + | | PlaceholderRowExec | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------------------------------------------------+ " ); @@ -3297,29 +3295,15 @@ async fn test_count_wildcard_on_aggregate() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+-----------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+-----------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-----------------------------------------------------------------------------------------------------------------------+ + +---------------+---------------------------------------------------------------+ + | plan_type | plan | + +---------------+---------------------------------------------------------------+ + | logical_plan | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t1 projection=[] | + | physical_plan | ProjectionExec: expr=[4 as count(*)] | + | | PlaceholderRowExec | + | | | + +---------------+---------------------------------------------------------------+ " ); @@ -3340,33 +3324,32 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | ProjectionExec: expr=[a@2 as a, b@3 as b, count(*)@0 as count(*), __always_true@1 as __always_true] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[count(*)@0, __always_true@2, a@3, b@4] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | ProjectionExec: expr=[a@2 as a, b@3 as b, count(*)@0 as count(*), __always_true@1 as __always_true] | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[count(*)@0, __always_true@2, a@3, b@4] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 7cc957b5d2e5..eb93dc40c0c6 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -818,8 +818,7 @@ async fn test_physical_plan_display_indent_multi_children() { DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true CoalesceBatchesExec: target_batch_size=4096 RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 - ProjectionExec: expr=[c1@0 as c2] - DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true + DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1@0 as c2], file_type=csv, has_header=true " ); } From f31c00e60f3cf660316e8ce58a57ecb36e21e4c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 27 Nov 2025 23:06:46 +0100 Subject: [PATCH 06/14] Clippy --- datafusion/physical-optimizer/src/coalesce_batches.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 25042a09be9c..af42de74d6cc 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -27,7 +27,7 @@ use datafusion_common::{assert_eq_or_internal_err, DataFusionError}; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::{ async_func::AsyncFuncExec, coalesce_batches::CoalesceBatchesExec, - joins::HashJoinExec, repartition::RepartitionExec, ExecutionPlan, + repartition::RepartitionExec, ExecutionPlan, }; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; From a35584e5410b9743634eb09a4c4d8f4b1c4512b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 27 Nov 2025 23:10:11 +0100 Subject: [PATCH 07/14] Clippy --- datafusion/core/tests/dataframe/mod.rs | 84 +++++++++++++------------- 1 file changed, 41 insertions(+), 43 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 5d4aee26d051..52b30326809e 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2996,22 +2996,21 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | - | | ProjectionExec: expr=[4 as count(*)] | - | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __correlated_sq_1 | + | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[] | + | physical_plan | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | + | | ProjectionExec: expr=[4 as count(*)] | + | | PlaceholderRowExec | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------------------------------------------------+ " ); @@ -3381,33 +3380,32 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | ProjectionExec: expr=[a@2 as a, b@3 as b, count(*)@0 as count(*), __always_true@1 as __always_true] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[count(*)@0, __always_true@2, a@3, b@4] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | ProjectionExec: expr=[a@2 as a, b@3 as b, count(*)@0 as count(*), __always_true@1 as __always_true] | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[count(*)@0, __always_true@2, a@3, b@4] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------------------------------------------------------+ " ); From 5c0f87d175dedf471b12de275bcb7963efc77ac2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 07:25:30 +0100 Subject: [PATCH 08/14] tests --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index b6a5e18da1f1..9d8b6e20a64d 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -975,6 +975,11 @@ impl JoinFuzzTestCase { .zip(&hj_formatted_sorted) .enumerate() { + if nlj_line == "+---+---+---+---+" && hj_line == "++" + || (nlj_line == "++" && hj_line == "+---+---+---+---+") + { + // both are empty results + } assert_eq!( (i, nlj_line), (i, hj_line), From 1c575001510faf423f305e7359141e06d627f5dc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 07:29:58 +0100 Subject: [PATCH 09/14] tests --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 9d8b6e20a64d..17c4fafffa61 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -975,8 +975,8 @@ impl JoinFuzzTestCase { .zip(&hj_formatted_sorted) .enumerate() { - if nlj_line == "+---+---+---+---+" && hj_line == "++" - || (nlj_line == "++" && hj_line == "+---+---+---+---+") + if nlj_line == &"+---+---+---+---+" && hj_line == &"++" + || (nlj_line == &"++" && hj_line == &"+---+---+---+---+") { // both are empty results } From 1f1940e031df1de336da89e271c161608d5e3d74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 07:41:25 +0100 Subject: [PATCH 10/14] Reusing buffers --- .../physical-plan/src/joins/hash_join/exec.rs | 8 ++++ .../src/joins/hash_join/stream.rs | 38 +++++++++++++---- .../physical-plan/src/joins/join_hash_map.rs | 41 ++++++++++++------- .../src/joins/stream_join_utils.rs | 6 ++- 4 files changed, 69 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 1bd18dce7bd2..b6c2130ab6c1 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -3487,6 +3487,8 @@ mod tests { let mut hashes_buffer = vec![0; right.num_rows()]; create_hashes([&right_keys_values], &random_state, &mut hashes_buffer)?; + let mut probe_indices_buffer = Vec::new(); + let mut build_indices_buffer = Vec::new(); let (l, r, _) = lookup_join_hashmap( &join_hash_map, &[left_keys_values], @@ -3495,6 +3497,8 @@ mod tests { &hashes_buffer, 8192, (0, None), + &mut probe_indices_buffer, + &mut build_indices_buffer, )?; let left_ids: UInt64Array = vec![0, 1].into(); @@ -3544,6 +3548,8 @@ mod tests { let mut hashes_buffer = vec![0; right.num_rows()]; create_hashes([&right_keys_values], &random_state, &mut hashes_buffer)?; + let mut probe_indices_buffer = Vec::new(); + let mut build_indices_buffer = Vec::new(); let (l, r, _) = lookup_join_hashmap( &join_hash_map, &[left_keys_values], @@ -3552,6 +3558,8 @@ mod tests { &hashes_buffer, 8192, (0, None), + &mut probe_indices_buffer, + &mut build_indices_buffer, )?; // We still expect to match rows 0 and 1 on both sides diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index 7d121b32c2b5..a9b0f5adb385 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -207,6 +207,10 @@ pub(super) struct HashJoinStream { batch_size: usize, /// Scratch space for computing hashes hashes_buffer: Vec, + /// Scratch space for probe indices during hash lookup + probe_indices_buffer: Vec, + /// Scratch space for build indices during hash lookup + build_indices_buffer: Vec, /// Specifies whether the right side has an ordering to potentially preserve right_side_ordered: bool, /// Shared build accumulator for coordinating dynamic filter updates (collects hash maps and/or bounds, optional) @@ -285,21 +289,35 @@ pub(super) fn lookup_join_hashmap( hashes_buffer: &[u64], limit: usize, offset: JoinHashMapOffset, + probe_indices_buffer: &mut Vec, + build_indices_buffer: &mut Vec, ) -> Result<(UInt64Array, UInt32Array, Option)> { - let (probe_indices, build_indices, next_offset) = - build_hashmap.get_matched_indices_with_limit_offset(hashes_buffer, limit, offset); - - let build_indices: UInt64Array = build_indices.into(); - let probe_indices: UInt32Array = probe_indices.into(); - + let next_offset = build_hashmap.get_matched_indices_with_limit_offset( + hashes_buffer, + limit, + offset, + probe_indices_buffer, + build_indices_buffer, + ); + + let build_indices_unfiltered: UInt64Array = + std::mem::take(build_indices_buffer).into(); + let probe_indices_unfiltered: UInt32Array = + std::mem::take(probe_indices_buffer).into(); + + // TODO: optimize equal_rows_arr to avoid allocation of new buffers let (build_indices, probe_indices) = equal_rows_arr( - &build_indices, - &probe_indices, + &build_indices_unfiltered, + &probe_indices_unfiltered, build_side_values, probe_side_values, null_equality, )?; + // Reclaim buffers + *build_indices_buffer = build_indices_unfiltered.into_parts().1.into(); + *probe_indices_buffer = probe_indices_unfiltered.into_parts().1.into(); + Ok((build_indices, probe_indices, next_offset)) } @@ -376,6 +394,8 @@ impl HashJoinStream { build_side, batch_size, hashes_buffer, + probe_indices_buffer: Vec::with_capacity(batch_size), + build_indices_buffer: Vec::with_capacity(batch_size), right_side_ordered, build_accumulator, build_waiter: None, @@ -581,6 +601,8 @@ impl HashJoinStream { &self.hashes_buffer, self.batch_size, state.offset, + &mut self.probe_indices_buffer, + &mut self.build_indices_buffer, )?; let distinct_right_indices_count = count_distinct_sorted_indices(&right_indices); diff --git a/datafusion/physical-plan/src/joins/join_hash_map.rs b/datafusion/physical-plan/src/joins/join_hash_map.rs index 0adc16450961..6c65b0359163 100644 --- a/datafusion/physical-plan/src/joins/join_hash_map.rs +++ b/datafusion/physical-plan/src/joins/join_hash_map.rs @@ -114,7 +114,9 @@ pub trait JoinHashMapType: Send + Sync { hash_values: &[u64], limit: usize, offset: JoinHashMapOffset, - ) -> (Vec, Vec, Option); + input_indices: &mut Vec, + match_indices: &mut Vec, + ) -> Option; /// Returns `true` if the join hash map contains no entries. fn is_empty(&self) -> bool; @@ -171,13 +173,17 @@ impl JoinHashMapType for JoinHashMapU32 { hash_values: &[u64], limit: usize, offset: JoinHashMapOffset, - ) -> (Vec, Vec, Option) { + input_indices: &mut Vec, + match_indices: &mut Vec, + ) -> Option { get_matched_indices_with_limit_offset::( &self.map, &self.next, hash_values, limit, offset, + input_indices, + match_indices, ) } @@ -237,13 +243,17 @@ impl JoinHashMapType for JoinHashMapU64 { hash_values: &[u64], limit: usize, offset: JoinHashMapOffset, - ) -> (Vec, Vec, Option) { + input_indices: &mut Vec, + match_indices: &mut Vec, + ) -> Option { get_matched_indices_with_limit_offset::( &self.map, &self.next, hash_values, limit, offset, + input_indices, + match_indices, ) } @@ -388,14 +398,16 @@ pub fn get_matched_indices_with_limit_offset( hash_values: &[u64], limit: usize, offset: JoinHashMapOffset, -) -> (Vec, Vec, Option) + input_indices: &mut Vec, + match_indices: &mut Vec, +) -> Option where T: Copy + TryFrom + PartialOrd + Into + Sub, >::Error: Debug, T: ArrowNativeType, { - let mut input_indices = Vec::with_capacity(limit); - let mut match_indices = Vec::with_capacity(limit); + input_indices.clear(); + match_indices.clear(); let one = T::try_from(1).unwrap(); // Check if hashmap consists of unique values @@ -409,12 +421,11 @@ where match_indices.push((*idx - one).into()); } } - let next_off = if end == hash_values.len() { + return if end == hash_values.len() { None } else { Some((end, None)) }; - return (input_indices, match_indices, next_off); } let mut remaining_output = limit; @@ -436,11 +447,11 @@ where idx, next_idx, &mut remaining_output, - &mut input_indices, - &mut match_indices, + input_indices, + match_indices, is_last, ) { - return (input_indices, match_indices, Some(next_offset)); + return Some(next_offset); } idx + 1 } @@ -457,13 +468,13 @@ where row_idx, idx, &mut remaining_output, - &mut input_indices, - &mut match_indices, + input_indices, + match_indices, is_last, ) { - return (input_indices, match_indices, Some(next_offset)); + return Some(next_offset); } } } - (input_indices, match_indices, None) + None } diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index f4a3cd92f16d..2d58f651a4dc 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -80,7 +80,9 @@ impl JoinHashMapType for PruningJoinHashMap { hash_values: &[u64], limit: usize, offset: JoinHashMapOffset, - ) -> (Vec, Vec, Option) { + input_indices: &mut Vec, + match_indices: &mut Vec, + ) -> Option { // Flatten the deque let next: Vec = self.next.iter().copied().collect(); get_matched_indices_with_limit_offset::( @@ -89,6 +91,8 @@ impl JoinHashMapType for PruningJoinHashMap { hash_values, limit, offset, + input_indices, + match_indices, ) } From 447a913d0af206bc116c58052f220f35785f40e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 08:13:11 +0100 Subject: [PATCH 11/14] test --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 17c4fafffa61..e2b0b1c6c222 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -975,10 +975,9 @@ impl JoinFuzzTestCase { .zip(&hj_formatted_sorted) .enumerate() { - if nlj_line == &"+---+---+---+---+" && hj_line == &"++" - || (nlj_line == &"++" && hj_line == &"+---+---+---+---+") - { + if !nlj_line.contains("\n") && !hj_line.contains("\n") { // both are empty results + continue; } assert_eq!( (i, nlj_line), From a4f8ad74bc4d227920766d802d177727e1a35a50 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 12:30:17 +0100 Subject: [PATCH 12/14] Feedback --- .../src/joins/hash_join/stream.rs | 4 +--- .../src/joins/nested_loop_join.rs | 18 +++++++++--------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index a9b0f5adb385..559362183244 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -438,9 +438,7 @@ impl HashJoinStream { HashJoinStreamState::Completed => { // Flush any remaining buffered data if !self.output_buffer.is_empty() { - self.output_buffer.finish_buffered_batch().map_err( - |e| -> datafusion_common::DataFusionError { e.into() }, - )?; + self.output_buffer.finish_buffered_batch()?; // Continue loop to emit the flushed batch continue; } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index f16e2220dfbe..02166b42710b 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -769,7 +769,7 @@ pub(crate) struct NestedLoopJoinStream { state: NLJState, /// Output buffer holds the join result to output. It will emit eagerly when /// the threshold is reached. - output_buffer: Box, + batch_coalescer: Box, /// See comments in [`NLJState::Done`] for its purpose handled_empty_output: bool, @@ -1034,7 +1034,7 @@ impl NestedLoopJoinStream { left_data, metrics, buffered_left_data: None, - output_buffer: Box::new(BatchCoalescer::new(schema, batch_size)), + batch_coalescer: Box::new(BatchCoalescer::new(schema, batch_size)), batch_size, current_right_batch: None, current_right_batch_matched: None, @@ -1174,7 +1174,7 @@ impl NestedLoopJoinStream { // Construct the result batch for unmatched right rows using a utility function match self.process_right_unmatched() { Ok(Some(batch)) => { - match self.output_buffer.push_batch(batch) { + match self.batch_coalescer.push_batch(batch) { Ok(()) => { // Processed all in one pass // cleared inside `process_right_unmatched` @@ -1212,7 +1212,7 @@ impl NestedLoopJoinStream { Ok(true) => ControlFlow::Continue(()), // To Done state // We have finished processing all unmatched rows - Ok(false) => match self.output_buffer.finish_buffered_batch() { + Ok(false) => match self.batch_coalescer.finish_buffered_batch() { Ok(()) => { self.state = NLJState::Done; ControlFlow::Continue(()) @@ -1303,7 +1303,7 @@ impl NestedLoopJoinStream { )?; if let Some(batch) = joined_batch { - self.output_buffer.push_batch(batch)?; + self.batch_coalescer.push_batch(batch)?; } self.left_probe_idx += l_row_count; @@ -1316,7 +1316,7 @@ impl NestedLoopJoinStream { self.process_single_left_row_join(&left_data, &right_batch, l_idx)?; if let Some(batch) = joined_batch { - self.output_buffer.push_batch(batch)?; + self.batch_coalescer.push_batch(batch)?; } // ==== Prepare for the next iteration ==== @@ -1607,7 +1607,7 @@ impl NestedLoopJoinStream { if let Some(batch) = self.process_left_unmatched_range(left_data, start_idx, end_idx)? { - self.output_buffer.push_batch(batch)?; + self.batch_coalescer.push_batch(batch)?; } // ==== Prepare for the next iteration ==== @@ -1712,8 +1712,8 @@ impl NestedLoopJoinStream { /// Flush the `output_buffer` if there are batches ready to output /// None if no result batch ready. fn maybe_flush_ready_batch(&mut self) -> Option>>> { - if self.output_buffer.has_completed_batch() { - if let Some(batch) = self.output_buffer.next_completed_batch() { + if self.batch_coalescer.has_completed_batch() { + if let Some(batch) = self.batch_coalescer.next_completed_batch() { // Update output rows for selectivity metric let output_rows = batch.num_rows(); self.metrics.selectivity.add_part(output_rows); From 5d69519b66e7950330184053a3d07248ff2326aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 12:31:48 +0100 Subject: [PATCH 13/14] Add back comment --- datafusion/physical-optimizer/src/coalesce_batches.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index af42de74d6cc..12996c8eb674 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -58,6 +58,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { plan.transform_up(|plan| { let plan_any = plan.as_any(); let wrap_in_coalesce = plan_any + // Don't need to add CoalesceBatchesExec after a round robin RepartitionExec .downcast_ref::() .map(|repart_exec| { !matches!( From 60bdf21d2f58e341427239b90537aeef77579544 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 28 Nov 2025 14:27:50 +0100 Subject: [PATCH 14/14] Fix tests --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index e2b0b1c6c222..86c933884098 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -966,6 +966,10 @@ impl JoinFuzzTestCase { if join_tests.contains(&NljHj) { let err_msg_rowcnt = format!("NestedLoopJoinExec and HashJoinExec produced different row counts, batch_size: {batch_size}"); assert_eq!(nlj_rows, hj_rows, "{}", err_msg_rowcnt.as_str()); + if nlj_rows == 0 && hj_rows == 0 { + // both joins returned no rows, skip content comparison + continue; + } let err_msg_contents = format!("NestedLoopJoinExec and HashJoinExec produced different results, batch_size: {batch_size}"); // row level compare if any of joins returns the result @@ -975,10 +979,6 @@ impl JoinFuzzTestCase { .zip(&hj_formatted_sorted) .enumerate() { - if !nlj_line.contains("\n") && !hj_line.contains("\n") { - // both are empty results - continue; - } assert_eq!( (i, nlj_line), (i, hj_line),