diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c09db371912b0..9adf766f8ffb1 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2507,13 +2507,9 @@ async fn verify_join_output_partitioning() -> Result<()> { | JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => { - let right_exprs: Vec> = vec![ - Arc::new(Column::new_with_schema("c2_c1", &join_schema)?), - Arc::new(Column::new_with_schema("c2_c2", &join_schema)?), - ]; assert_eq!( out_partitioning, - &Partitioning::Hash(right_exprs, default_partition_count) + &Partitioning::RoundRobinBatch(default_partition_count) ); } JoinType::Full => { diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 94ae82a9ad755..78ea0e7c52cdd 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -708,15 +708,15 @@ fn multi_hash_joins() -> Result<()> { // Should include 4 RepartitionExecs _ => { assert_plan!(plan_distrib, @r" - HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@6, c@2)] - RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=1 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -776,13 +776,14 @@ fn multi_joins_after_alias() -> Result<()> { plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)] - ProjectionExec: expr=[a@0 as a1, a@0 as a2] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 + ProjectionExec: expr=[a@0 as a1, a@0 as a2] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); @@ -804,13 +805,14 @@ fn multi_joins_after_alias() -> Result<()> { plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)] - ProjectionExec: expr=[a@0 as a1, a@0 as a2] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] - RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + RepartitionExec: partitioning=Hash([a2@1], 10), input_partitions=10 + ProjectionExec: expr=[a@0 as a1, a@0 as a2] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); @@ -863,9 +865,9 @@ fn multi_joins_after_multi_alias() -> Result<()> { HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=1 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=1 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); @@ -1085,17 +1087,18 @@ fn multi_hash_join_key_ordering() -> Result<()> { @r" FilterExec: c@6 > 1 HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)] - ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + RepartitionExec: partitioning=Hash([B@2, C@3, AA@1], 10), input_partitions=10 + ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=1 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " @@ -1221,21 +1224,21 @@ fn reorder_join_keys_to_left_input() -> Result<()> { assert_eq!(captured_join_type, join_type.to_string()); insta::allow_duplicates! {insta::assert_snapshot!(modified_plan, @r" - HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)] - ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] - RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - ");} + HashJoinExec: mode=Partitioned, join_type=..., on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)] + ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] + RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + ");} } Ok(()) @@ -1349,21 +1352,21 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); insta::allow_duplicates! {insta::assert_snapshot!(plan_str, @r" - HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)] - ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)] - RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] - RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=1 - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=1 - ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] - DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - ");} + HashJoinExec: mode=Partitioned, join_type=..., on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)] + ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)] + RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + ");} } Ok(()) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index d265841246867..8fce653a8bb75 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1610,7 +1610,7 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab AND b@0 IN (SET) ([aa, ab]) ] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb AND d@0 IN (SET) ([ca, cb]) ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= ce AND d@0 IN (SET) ([ca, cb, cc, cd, ce]) ] " ); } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index fa248c448683b..c242acd0c8744 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -804,7 +804,7 @@ async fn test_physical_plan_display_indent_multi_children() { HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0] 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 - RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=1 + RepartitionExec: partitioning=RoundRobinBatch(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-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 988e14c28e17c..8aab3fd2ff0a6 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -58,7 +58,7 @@ pub use analysis::{AnalysisContext, ExprBoundaries, analyze}; pub use equivalence::{ AcrossPartitions, ConstExpr, EquivalenceProperties, calculate_union, }; -pub use partitioning::{Distribution, Partitioning}; +pub use partitioning::{Distribution, Partitioning, PartitioningSatisfaction}; pub use physical_expr::{ add_offset_to_expr, add_offset_to_physical_sort_exprs, create_lex_ordering, create_ordering, create_physical_sort_expr, create_physical_sort_exprs, diff --git a/datafusion/physical-plan/src/joins/array_map.rs b/datafusion/physical-plan/src/joins/array_map.rs index ad40d6776df4f..31a6f39fd8a1d 100644 --- a/datafusion/physical-plan/src/joins/array_map.rs +++ b/datafusion/physical-plan/src/joins/array_map.rs @@ -238,6 +238,7 @@ impl ArrayMap { pub fn get_matched_indices_with_limit_offset( &self, prob_side_keys: &[ArrayRef], + indices: Option<&[u32]>, limit: usize, current_offset: MapOffset, probe_indices: &mut Vec, @@ -256,6 +257,7 @@ impl ArrayMap { lookup_and_get_indices, self, array, + indices, limit, current_offset, probe_indices, @@ -267,6 +269,7 @@ impl ArrayMap { fn lookup_and_get_indices( &self, array: &ArrayRef, + indices: Option<&[u32]>, limit: usize, current_offset: MapOffset, probe_indices: &mut Vec, @@ -283,11 +286,14 @@ impl ArrayMap { let have_null = arr.null_count() > 0; if self.next.is_empty() { - for prob_idx in current_offset.0..arr.len() { + let num_rows = indices.map_or(arr.len(), |v| v.len()); + for i in current_offset.0..num_rows { if build_indices.len() == limit { - return Ok(Some((prob_idx, None))); + return Ok(Some((i, None))); } + let prob_idx = indices.map_or(i, |v| v[i] as usize); + // short circuit if have_null && arr.is_null(prob_idx) { continue; @@ -307,6 +313,7 @@ impl ArrayMap { Ok(None) } else { let mut remaining_output = limit; + let num_rows = indices.map_or(arr.len(), |v| v.len()); let to_skip = match current_offset { // None `initial_next_idx` indicates that `initial_idx` processing hasn't been started (idx, None) => idx, @@ -316,35 +323,38 @@ impl ArrayMap { // Otherwise, process remaining `initial_idx` matches by traversing `next_chain`, // to start with the next index (idx, Some(next_idx)) => { - let is_last = idx == arr.len() - 1; + let prob_idx = indices.map_or(idx, |v| v[idx] as usize); + let is_last = idx == num_rows - 1; if let Some(next_offset) = traverse_chain( &self.next, - idx, + prob_idx, next_idx as u32, &mut remaining_output, probe_indices, build_indices, is_last, ) { - return Ok(Some(next_offset)); + return Ok(Some((idx, next_offset.1))); } idx + 1 } }; - for prob_side_idx in to_skip..arr.len() { + for i in to_skip..num_rows { if remaining_output == 0 { - return Ok(Some((prob_side_idx, None))); + return Ok(Some((i, None))); } - if arr.is_null(prob_side_idx) { + let prob_idx = indices.map_or(i, |v| v[i] as usize); + + if arr.is_null(prob_idx) { continue; } - let is_last = prob_side_idx == arr.len() - 1; + let is_last = i == num_rows - 1; // SAFETY: prob_idx is guaranteed to be within bounds by the loop range. - let prob_val: u64 = unsafe { arr.value_unchecked(prob_side_idx) }.as_(); + let prob_val: u64 = unsafe { arr.value_unchecked(prob_idx) }.as_(); let idx_in_build_side = prob_val.wrapping_sub(self.offset) as usize; if idx_in_build_side >= self.data.len() || self.data[idx_in_build_side] == 0 @@ -356,14 +366,14 @@ impl ArrayMap { if let Some(offset) = traverse_chain( &self.next, - prob_side_idx, + prob_idx, build_idx, &mut remaining_output, probe_indices, build_indices, is_last, ) { - return Ok(Some(offset)); + return Ok(Some((i, offset.1))); } } Ok(None) @@ -430,6 +440,7 @@ mod tests { while let Some(o) = next { next = map.get_matched_indices_with_limit_offset( &probe, + None, 1, o, &mut prob_idx, @@ -457,6 +468,7 @@ mod tests { // Skip 10, find 1, next is 2 let next = map.get_matched_indices_with_limit_offset( &probe, + None, 1, (0, None), &mut p_idx, @@ -469,6 +481,7 @@ mod tests { // Find 2, end let next = map.get_matched_indices_with_limit_offset( &probe, + None, 1, next.unwrap(), &mut p_idx, @@ -494,6 +507,7 @@ mod tests { // batch_size=3, should get 2 matches from first '1' and 1 match from second '1' let result_offset = array_map.get_matched_indices_with_limit_offset( &prob_side_keys, + None, 3, (0, None), &mut prob_indices, @@ -525,6 +539,7 @@ mod tests { // Call once to get all matches let result_offset = array_map.get_matched_indices_with_limit_offset( &prob_side_keys, + None, 10, // A batch size larger than number of probes (0, None), &mut prob_indices, diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 131b07461ebe5..d1305163649b3 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -38,7 +38,7 @@ use crate::joins::hash_join::stream::{ }; use crate::joins::join_hash_map::{JoinHashMapU32, JoinHashMapU64}; use crate::joins::utils::{ - OnceAsync, OnceFut, asymmetric_join_output_partitioning, reorder_output_after_swap, + OnceAsync, asymmetric_join_output_partitioning, reorder_output_after_swap, swap_join_projection, update_hash, }; use crate::joins::{JoinOn, JoinOnRef, PartitionMode, SharedBitmapBuilder}; @@ -51,7 +51,7 @@ use crate::repartition::REPARTITION_RANDOM_STATE; use crate::spill::get_record_batch_memory_size; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, + PartitioningSatisfaction, PlanProperties, SendableRecordBatchStream, Statistics, common::can_project, joins::utils::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinHashMapType, @@ -70,9 +70,10 @@ use arrow_schema::DataType; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::{ - JoinSide, JoinType, NullEquality, Result, assert_or_internal_err, internal_err, - plan_err, project_schema, + DataFusionError, JoinSide, JoinType, NullEquality, Result, assert_or_internal_err, + internal_err, plan_err, project_schema, }; +use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_expr::Accumulator; @@ -91,9 +92,8 @@ use parking_lot::Mutex; use super::partitioned_hash_eval::SeededRandomState; -/// Hard-coded seed to ensure hash values from the hash join differ from `RepartitionExec`, avoiding collisions. -pub(crate) const HASH_JOIN_SEED: SeededRandomState = - SeededRandomState::with_seeds('J' as u64, 'O' as u64, 'I' as u64, 'N' as u64); +/// Hard-coded seed to ensure hash values from the hash join match `RepartitionExec`'s hashing for internal routing. +pub(crate) const HASH_JOIN_SEED: SeededRandomState = REPARTITION_RANDOM_STATE; const ARRAY_MAP_CREATED_COUNT_METRIC_NAME: &str = "array_map_created_count"; @@ -208,9 +208,9 @@ pub(super) struct JoinLeftData { pub(super) membership: PushdownStrategy, /// Shared atomic flag indicating if any probe partition saw data (for null-aware anti joins) /// This is shared across all probe partitions to provide global knowledge - pub(super) probe_side_non_empty: AtomicBool, + pub(super) probe_side_non_empty: Arc, /// Shared atomic flag indicating if any probe partition saw NULL in join keys (for null-aware anti joins) - pub(super) probe_side_has_null: AtomicBool, + pub(super) probe_side_has_null: Arc, } impl JoinLeftData { @@ -452,13 +452,13 @@ pub struct HashJoinExec { /// The schema after join. Please be careful when using this schema, /// if there is a projection, the schema isn't the same as the output schema. join_schema: SchemaRef, - /// Future that consumes left input and builds the hash table + /// Futures that consume left input and build the hash tables. /// - /// For CollectLeft partition mode, this structure is *shared* across all output streams. + /// For both CollectLeft and Partitioned partition modes, this structure is *shared* across all output streams. /// - /// Each output stream waits on the `OnceAsync` to signal the completion of - /// the hash table creation. - left_fut: Arc>, + /// Each output stream waits on these `OnceAsync`s to signal the completion of + /// the hash table creation for the relevant partitions. + left_futs: Vec>>, /// Shared the `SeededRandomState` for the hashing algorithm (seeds preserved for serialization) random_state: SeededRandomState, /// Partitioning mode to use @@ -479,6 +479,10 @@ pub struct HashJoinExec { /// Set when dynamic filter pushdown is detected in handle_child_pushdown_result. /// HashJoinExec also needs to keep a shared bounds accumulator for coordinating updates. dynamic_filter: Option, + /// Shared atomic flag indicating if any probe partition saw data (for null-aware anti joins) + probe_side_non_empty: Arc, + /// Shared atomic flag indicating if any probe partition saw NULL in join keys (for null-aware anti joins) + probe_side_has_null: Arc, } #[derive(Clone)] @@ -499,7 +503,7 @@ impl fmt::Debug for HashJoinExec { .field("filter", &self.filter) .field("join_type", &self.join_type) .field("join_schema", &self.join_schema) - .field("left_fut", &self.left_fut) + .field("left_futs", &self.left_futs) .field("random_state", &self.random_state) .field("mode", &self.mode) .field("metrics", &self.metrics) @@ -581,6 +585,16 @@ impl HashJoinExec { // Initialize both dynamic filter and bounds accumulator to None // They will be set later if dynamic filtering is enabled + let num_futs = if partition_mode == PartitionMode::CollectLeft { + 1 + } else { + left.output_partitioning().partition_count() + }; + + let left_futs = (0..num_futs) + .map(|_| Arc::new(OnceAsync::default())) + .collect(); + Ok(HashJoinExec { left, right, @@ -588,7 +602,7 @@ impl HashJoinExec { filter, join_type: *join_type, join_schema, - left_fut: Default::default(), + left_futs, random_state, mode: partition_mode, metrics: ExecutionPlanMetricsSet::new(), @@ -598,6 +612,8 @@ impl HashJoinExec { null_aware, cache, dynamic_filter: None, + probe_side_non_empty: Arc::new(AtomicBool::new(false)), + probe_side_has_null: Arc::new(AtomicBool::new(false)), }) } @@ -948,14 +964,14 @@ impl ExecutionPlan for HashJoinExec { Distribution::UnspecifiedDistribution, ], PartitionMode::Partitioned => { - let (left_expr, right_expr) = self + let (left_expr, _): (Vec<_>, Vec<_>) = self .on .iter() .map(|(l, r)| (Arc::clone(l), Arc::clone(r))) .unzip(); vec![ Distribution::HashPartitioned(left_expr), - Distribution::HashPartitioned(right_expr), + Distribution::UnspecifiedDistribution, ] } PartitionMode::Auto => vec![ @@ -1005,7 +1021,13 @@ impl ExecutionPlan for HashJoinExec { filter: self.filter.clone(), join_type: self.join_type, join_schema: Arc::clone(&self.join_schema), - left_fut: Arc::clone(&self.left_fut), + left_futs: (0..if self.mode == PartitionMode::CollectLeft { + 1 + } else { + children[0].output_partitioning().partition_count() + }) + .map(|_| Arc::new(OnceAsync::default())) + .collect(), random_state: self.random_state.clone(), mode: self.mode, metrics: ExecutionPlanMetricsSet::new(), @@ -1013,6 +1035,8 @@ impl ExecutionPlan for HashJoinExec { column_indices: self.column_indices.clone(), null_equality: self.null_equality, null_aware: self.null_aware, + probe_side_non_empty: Arc::new(AtomicBool::new(false)), + probe_side_has_null: Arc::new(AtomicBool::new(false)), cache: Self::compute_properties( &children[0], &children[1], @@ -1035,8 +1059,14 @@ impl ExecutionPlan for HashJoinExec { filter: self.filter.clone(), join_type: self.join_type, join_schema: Arc::clone(&self.join_schema), - // Reset the left_fut to allow re-execution - left_fut: Arc::new(OnceAsync::default()), + // Reset the left_futs to allow re-execution + left_futs: (0..if self.mode == PartitionMode::CollectLeft { + 1 + } else { + self.left.output_partitioning().partition_count() + }) + .map(|_| Arc::new(OnceAsync::default())) + .collect(), random_state: self.random_state.clone(), mode: self.mode, metrics: ExecutionPlanMetricsSet::new(), @@ -1044,6 +1074,8 @@ impl ExecutionPlan for HashJoinExec { column_indices: self.column_indices.clone(), null_equality: self.null_equality, null_aware: self.null_aware, + probe_side_non_empty: Arc::new(AtomicBool::new(false)), + probe_side_has_null: Arc::new(AtomicBool::new(false)), cache: self.cache.clone(), // Reset dynamic filter and bounds accumulator to initial state dynamic_filter: None, @@ -1061,14 +1093,6 @@ impl ExecutionPlan for HashJoinExec { .map(|on| Arc::clone(&on.0)) .collect::>(); let left_partitions = self.left.output_partitioning().partition_count(); - let right_partitions = self.right.output_partitioning().partition_count(); - - assert_or_internal_err!( - self.mode != PartitionMode::Partitioned - || left_partitions == right_partitions, - "Invalid HashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\ - consider using RepartitionExec" - ); assert_or_internal_err!( self.mode != PartitionMode::CollectLeft || left_partitions == 1, @@ -1094,58 +1118,54 @@ impl ExecutionPlan for HashJoinExec { let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics); - let array_map_created_count = MetricBuilder::new(&self.metrics) - .counter(ARRAY_MAP_CREATED_COUNT_METRIC_NAME, partition); - - let left_fut = match self.mode { - PartitionMode::CollectLeft => self.left_fut.try_once(|| { - let left_stream = self.left.execute(0, Arc::clone(&context))?; - - let reservation = - MemoryConsumer::new("HashJoinInput").register(context.memory_pool()); - - Ok(collect_left_input( - self.random_state.random_state().clone(), - left_stream, - on_left.clone(), - join_metrics.clone(), - reservation, - need_produce_result_in_final(self.join_type), - self.right().output_partitioning().partition_count(), - enable_dynamic_filter_pushdown, - Arc::clone(context.session_config().options()), - self.null_equality, - array_map_created_count, - )) - })?, - PartitionMode::Partitioned => { - let left_stream = self.left.execute(partition, Arc::clone(&context))?; - - let reservation = - MemoryConsumer::new(format!("HashJoinInput[{partition}]")) + let left_futs = (0..left_partitions) + .map(|i| { + let array_map_created_count = MetricBuilder::new(&self.metrics) + .counter(ARRAY_MAP_CREATED_COUNT_METRIC_NAME, i); + + let random_state = self.random_state.random_state().clone(); + let on_left = on_left.clone(); + let join_metrics = join_metrics.clone(); + let join_type = self.join_type; + let right_partition_count = + self.right().output_partitioning().partition_count(); + let null_equality = self.null_equality; + let probe_side_non_empty = Arc::clone(&self.probe_side_non_empty); + let probe_side_has_null = Arc::clone(&self.probe_side_has_null); + let options = Arc::clone(context.session_config().options()); + + self.left_futs[i].try_once(|| { + let left_stream = self.left.execute(i, Arc::clone(&context))?; + + let reservation = MemoryConsumer::new(format!("HashJoinInput[{i}]")) .register(context.memory_pool()); - OnceFut::new(collect_left_input( - self.random_state.random_state().clone(), - left_stream, - on_left.clone(), - join_metrics.clone(), - reservation, - need_produce_result_in_final(self.join_type), - 1, - enable_dynamic_filter_pushdown, - Arc::clone(context.session_config().options()), - self.null_equality, - array_map_created_count, - )) - } - PartitionMode::Auto => { - return plan_err!( - "Invalid HashJoinExec, unsupported PartitionMode {:?} in execute()", - PartitionMode::Auto - ); - } - }; + let task = SpawnedTask::spawn(collect_left_input( + random_state, + left_stream, + on_left, + join_metrics, + reservation, + need_produce_result_in_final(join_type), + right_partition_count, + enable_dynamic_filter_pushdown, + options, + null_equality, + array_map_created_count, + probe_side_non_empty, + probe_side_has_null, + )); + + Ok(Box::pin(async move { + task.join().await.map_err(|e| { + DataFusionError::Execution(format!( + "HashJoin build side task failed: {e}" + )) + })? + })) + }) + }) + .collect::>>()?; let batch_size = context.session_config().batch_size(); @@ -1195,8 +1215,21 @@ impl ExecutionPlan for HashJoinExec { .map(|(_, right_expr)| Arc::clone(right_expr)) .collect::>(); + let probe_side_partitioned = if self.mode == PartitionMode::Partitioned { + self.right.output_partitioning().partition_count() == left_partitions + && matches!( + self.right.output_partitioning().satisfaction( + &Distribution::HashPartitioned(on_right.clone()), + self.right.properties().equivalence_properties(), + false, + ), + PartitioningSatisfaction::Exact | PartitioningSatisfaction::Subset + ) + } else { + false + }; + Ok(Box::pin(HashJoinStream::new( - partition, self.schema(), on_right, self.filter.clone(), @@ -1207,11 +1240,11 @@ impl ExecutionPlan for HashJoinExec { column_indices_after_projection, self.null_equality, HashJoinStreamState::WaitBuildSide, - BuildSide::Initial(BuildSideInitialState { left_fut }), + BuildSide::Initial(BuildSideInitialState { left_futs }), batch_size, - vec![], - self.right.output_ordering().is_some(), build_accumulator, + partition, + probe_side_partitioned, self.mode, self.null_aware, ))) @@ -1365,7 +1398,7 @@ impl ExecutionPlan for HashJoinExec { filter: self.filter.clone(), join_type: self.join_type, join_schema: Arc::clone(&self.join_schema), - left_fut: Arc::clone(&self.left_fut), + left_futs: self.left_futs.clone(), random_state: self.random_state.clone(), mode: self.mode, metrics: ExecutionPlanMetricsSet::new(), @@ -1373,6 +1406,8 @@ impl ExecutionPlan for HashJoinExec { column_indices: self.column_indices.clone(), null_equality: self.null_equality, null_aware: self.null_aware, + probe_side_non_empty: Arc::clone(&self.probe_side_non_empty), + probe_side_has_null: Arc::clone(&self.probe_side_has_null), cache: self.cache.clone(), dynamic_filter: Some(HashJoinExecDynamicFilter { filter: dynamic_filter, @@ -1555,6 +1590,8 @@ async fn collect_left_input( config: Arc, null_equality: NullEquality, array_map_created_count: Count, + probe_side_non_empty: Arc, + probe_side_has_null: Arc, ) -> Result { let schema = left_stream.schema(); @@ -1737,8 +1774,8 @@ async fn collect_left_input( _reservation: reservation, bounds, membership, - probe_side_non_empty: AtomicBool::new(false), - probe_side_has_null: AtomicBool::new(false), + probe_side_non_empty, + probe_side_has_null, }; Ok(data) @@ -3995,6 +4032,7 @@ mod tests { &[right_keys_values], NullEquality::NullEqualsNothing, &hashes_buffer, + None, 8192, (0, None), &mut probe_indices_buffer, @@ -4056,6 +4094,7 @@ mod tests { &[right_keys_values], NullEquality::NullEqualsNothing, &hashes_buffer, + None, 8192, (0, None), &mut probe_indices_buffer, @@ -4829,12 +4868,12 @@ mod tests { // Asserting that operator-level reservation attempting to overallocate assert_contains!( err.to_string(), - "Resources exhausted: Additional allocation failed for HashJoinInput with top memory consumers (across reservations) as:\n HashJoinInput" + "Resources exhausted: Additional allocation failed for HashJoinInput[0] with top memory consumers (across reservations) as:\n HashJoinInput[0]" ); assert_contains!( err.to_string(), - "Failed to allocate additional 120.0 B for HashJoinInput" + "Failed to allocate additional 120.0 B for HashJoinInput[0]" ); } @@ -4911,12 +4950,12 @@ mod tests { // Asserting that stream-level reservation attempting to overallocate assert_contains!( err.to_string(), - "Resources exhausted: Additional allocation failed for HashJoinInput[1] with top memory consumers (across reservations) as:\n HashJoinInput[1]" + "Resources exhausted: Additional allocation failed for HashJoinInput" ); assert_contains!( err.to_string(), - "Failed to allocate additional 120.0 B for HashJoinInput[1]" + "Failed to allocate additional 120.0 B for HashJoinInput" ); } diff --git a/datafusion/physical-plan/src/joins/hash_join/stream.rs b/datafusion/physical-plan/src/joins/hash_join/stream.rs index 54e620f99de7a..6420779b209ec 100644 --- a/datafusion/physical-plan/src/joins/hash_join/stream.rs +++ b/datafusion/physical-plan/src/joins/hash_join/stream.rs @@ -39,19 +39,18 @@ use crate::{ hash_utils::create_hashes, joins::utils::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinHashMapType, - StatefulStreamResult, adjust_indices_by_join_type, apply_join_filter_to_indices, - build_batch_empty_build_side, build_batch_from_indices, + StatefulStreamResult, apply_join_filter_to_indices, build_batch_from_indices, need_produce_result_in_final, }, }; -use arrow::array::{Array, ArrayRef, UInt32Array, UInt64Array}; -use arrow::compute::BatchCoalescer; +use arrow::array::{ + Array, ArrayRef, BooleanBuilder, UInt32Array, UInt64Array, new_null_array, +}; +use arrow::compute::{BatchCoalescer, take}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::{ - JoinSide, JoinType, NullEquality, Result, internal_datafusion_err, internal_err, -}; +use datafusion_common::{JoinSide, JoinType, NullEquality, Result, internal_err}; use datafusion_physical_expr::PhysicalExprRef; use ahash::RandomState; @@ -68,14 +67,14 @@ pub(super) enum BuildSide { /// Container for BuildSide::Initial related data pub(super) struct BuildSideInitialState { - /// Future for building hash table from build-side input - pub(super) left_fut: OnceFut, + /// Futures for building hash tables from build-side input partitions + pub(super) left_futs: Vec>, } /// Container for BuildSide::Ready related data pub(super) struct BuildSideReadyState { - /// Collected build-side data - left_data: Arc, + /// Collected build-side data for each partition + left_data: Vec>, } impl BuildSide { @@ -155,20 +154,21 @@ pub(super) struct ProcessProbeBatchState { batch: RecordBatch, /// Probe-side on expressions values values: Vec, + /// Cached hashes for the probe batch + hashes: Vec, + /// Indices of probe rows sharded by build partition + partition_indices: Option>>, + /// Current build partition index we are processing + current_partition_idx: usize, /// Starting offset for JoinHashMap lookups offset: MapOffset, /// Max joined probe-side index from current batch joined_probe_idx: Option, + /// Track which probe rows have matched any build partition + probe_matched: Vec, } -impl ProcessProbeBatchState { - fn advance(&mut self, offset: MapOffset, joined_probe_idx: Option) { - self.offset = offset; - if joined_probe_idx.is_some() { - self.joined_probe_idx = joined_probe_idx; - } - } -} +impl ProcessProbeBatchState {} /// [`Stream`] for [`super::HashJoinExec`] that does the actual join. /// @@ -180,8 +180,6 @@ impl ProcessProbeBatchState { /// - Producing joined [`RecordBatch`]es incrementally /// - Emitting unmatched rows for outer/semi/anti joins in the final stage pub(super) struct HashJoinStream { - /// Partition identifier for debugging and determinism - partition: usize, /// Input schema schema: Arc, /// equijoin columns from the right (probe side) @@ -206,19 +204,21 @@ pub(super) struct HashJoinStream { build_side: BuildSide, /// Maximum output batch size 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, + /// Scratch space for hashes during join + hashes_buffer: Vec, /// Shared build accumulator for coordinating dynamic filter updates (collects hash maps and/or bounds, optional) build_accumulator: Option>, /// Optional future to signal when build information has been reported by all partitions /// and the dynamic filter has been updated build_waiter: Option>, + /// Partition index of this stream + partition: usize, + /// Whether the probe side is already partitioned + probe_side_partitioned: bool, /// Partitioning mode to use mode: PartitionMode, /// Output buffer for coalescing small batches into larger ones. @@ -290,6 +290,7 @@ pub(super) fn lookup_join_hashmap( probe_side_values: &[ArrayRef], null_equality: NullEquality, hashes_buffer: &[u64], + indices: Option<&[u32]>, limit: usize, offset: MapOffset, probe_indices_buffer: &mut Vec, @@ -297,6 +298,7 @@ pub(super) fn lookup_join_hashmap( ) -> Result<(UInt64Array, UInt32Array, Option)> { let next_offset = build_hashmap.get_matched_indices_with_limit_offset( hashes_buffer, + indices, limit, offset, probe_indices_buffer, @@ -328,36 +330,9 @@ pub(super) fn lookup_join_hashmap( /// Counts the number of distinct elements in the input array. /// /// The input array must be sorted (e.g., `[0, 1, 1, 2, 2, ...]`) and contain no null values. -#[inline] -fn count_distinct_sorted_indices(indices: &UInt32Array) -> usize { - if indices.is_empty() { - return 0; - } - - debug_assert!(indices.null_count() == 0); - - let values_buf = indices.values(); - let values = values_buf.as_ref(); - let mut iter = values.iter(); - let Some(&first) = iter.next() else { - return 0; - }; - - let mut count = 1usize; - let mut last = first; - for &value in iter { - if value != last { - last = value; - count += 1; - } - } - count -} - impl HashJoinStream { #[expect(clippy::too_many_arguments)] pub(super) fn new( - partition: usize, schema: Arc, on_right: Vec, filter: Option, @@ -370,9 +345,9 @@ impl HashJoinStream { state: HashJoinStreamState, build_side: BuildSide, batch_size: usize, - hashes_buffer: Vec, - right_side_ordered: bool, build_accumulator: Option>, + partition: usize, + probe_side_partitioned: bool, mode: PartitionMode, null_aware: bool, ) -> Self { @@ -385,7 +360,6 @@ impl HashJoinStream { ); Self { - partition, schema, on_right, filter, @@ -398,12 +372,13 @@ impl HashJoinStream { state, build_side, batch_size, - hashes_buffer, + hashes_buffer: Vec::with_capacity(batch_size), probe_indices_buffer: Vec::with_capacity(batch_size), build_indices_buffer: Vec::with_capacity(batch_size), - right_side_ordered, build_accumulator, build_waiter: None, + partition, + probe_side_partitioned, mode, output_buffer, null_aware, @@ -481,61 +456,54 @@ impl HashJoinStream { ) -> Poll>>> { let build_timer = self.join_metrics.build_time.timer(); // build hash table from left (build) side, if not yet done - let left_data = ready!( - self.build_side - .try_as_initial_mut()? - .left_fut - .get_shared(cx) - )?; + let initial_state = self.build_side.try_as_initial_mut()?; + let mut left_data = Vec::with_capacity(initial_state.left_futs.len()); + for fut in &mut initial_state.left_futs { + left_data.push(ready!(fut.get_shared(cx))?); + } build_timer.done(); - // Note: For null-aware anti join, we need to check the probe side (right) for NULLs, - // not the build side (left). The probe-side NULL check happens during process_probe_batch. - // The probe_side_has_null flag will be set there if any probe batch contains NULL. - // Handle dynamic filter build-side information accumulation - // - // Dynamic filter coordination between partitions: - // Report hash maps (Partitioned mode) or bounds (CollectLeft mode) to the accumulator - // which will handle synchronization and filter updates if let Some(ref build_accumulator) = self.build_accumulator { let build_accumulator = Arc::clone(build_accumulator); - - let left_side_partition_id = match self.mode { - PartitionMode::Partitioned => self.partition, - PartitionMode::CollectLeft => 0, - PartitionMode::Auto => unreachable!( - "PartitionMode::Auto should not be present at execution time. This is a bug in DataFusion, please report it!" - ), - }; - - // Determine pushdown strategy based on availability of InList values - let pushdown = left_data.membership().clone(); - - // Construct the appropriate build data enum variant based on partition mode - let build_data = match self.mode { - PartitionMode::Partitioned => PartitionBuildData::Partitioned { - partition_id: left_side_partition_id, - pushdown, - bounds: left_data - .bounds - .clone() - .unwrap_or_else(|| PartitionBounds::new(vec![])), - }, - PartitionMode::CollectLeft => PartitionBuildData::CollectLeft { - pushdown, - bounds: left_data - .bounds - .clone() - .unwrap_or_else(|| PartitionBounds::new(vec![])), - }, - PartitionMode::Auto => unreachable!( - "PartitionMode::Auto should not be present at execution time" - ), - }; - + let mut report_futs = Vec::new(); + for (i, data) in left_data.iter().enumerate() { + let left_side_partition_id = match self.mode { + PartitionMode::Partitioned => i, + PartitionMode::CollectLeft => 0, + _ => unreachable!(), + }; + + let pushdown = data.membership().clone(); + let build_data = match self.mode { + PartitionMode::Partitioned => PartitionBuildData::Partitioned { + partition_id: left_side_partition_id, + pushdown, + bounds: data + .bounds + .clone() + .unwrap_or_else(|| PartitionBounds::new(vec![])), + }, + PartitionMode::CollectLeft => PartitionBuildData::CollectLeft { + pushdown, + bounds: data + .bounds + .clone() + .unwrap_or_else(|| PartitionBounds::new(vec![])), + }, + _ => unreachable!(), + }; + + let build_accumulator_clone = Arc::clone(&build_accumulator); + report_futs.push(async move { + build_accumulator_clone.report_build_data(build_data).await + }); + } self.build_waiter = Some(OnceFut::new(async move { - build_accumulator.report_build_data(build_data).await + for fut in report_futs { + fut.await?; + } + Ok(()) })); self.state = HashJoinStreamState::WaitPartitionBoundsReport; } else { @@ -562,25 +530,46 @@ impl HashJoinStream { // Precalculate hash values for fetched batch let keys_values = evaluate_expressions_to_arrays(&self.on_right, &batch)?; - if let Map::HashMap(_) = self.build_side.try_as_ready()?.left_data.map() { - self.hashes_buffer.clear(); - self.hashes_buffer.resize(batch.num_rows(), 0); - create_hashes( - &keys_values, - &self.random_state, - &mut self.hashes_buffer, - )?; - } + let build_side = self.build_side.try_as_ready()?; + let num_parts = build_side.left_data.len(); + + self.hashes_buffer.clear(); + self.hashes_buffer.resize(batch.num_rows(), 0); + create_hashes(&keys_values, &self.random_state, &mut self.hashes_buffer)?; + let hashes = self.hashes_buffer.clone(); + + let partition_indices = if num_parts > 1 && !self.probe_side_partitioned { + let mut indices = + vec![Vec::with_capacity(batch.num_rows() / num_parts); num_parts]; + for (i, hash) in self.hashes_buffer.iter().enumerate() { + let p = (hash % num_parts as u64) as usize; + indices[p].push(i as u32); + } + Some(indices) + } else { + None + }; + + let current_partition_idx = if self.probe_side_partitioned { + self.partition + } else { + 0 + }; self.join_metrics.input_batches.add(1); self.join_metrics.input_rows.add(batch.num_rows()); + let num_rows = batch.num_rows(); self.state = HashJoinStreamState::ProcessProbeBatch(ProcessProbeBatchState { batch, values: keys_values, + hashes, + partition_indices, + current_partition_idx, offset: (0, None), joined_probe_idx: None, + probe_matched: vec![false; num_rows], }); } Some(Err(err)) => return Poll::Ready(Err(err)), @@ -604,35 +593,22 @@ impl HashJoinStream { let timer = self.join_metrics.join_time.timer(); - // Null-aware anti join semantics: - // For LeftAnti: output LEFT (build) rows where LEFT.key NOT IN RIGHT.key - // 1. If RIGHT (probe) contains NULL in any batch, no LEFT rows should be output - // 2. LEFT rows with NULL keys should not be output (handled in final stage) + // Null-aware anti join semantics if self.null_aware { - // Mark that we've seen a probe batch with actual rows (probe side is non-empty) - // Only set this if batch has rows - empty batches don't count - // Use shared atomic state so all partitions can see this global information if state.batch.num_rows() > 0 { - build_side - .left_data + build_side.left_data[0] .probe_side_non_empty .store(true, Ordering::Relaxed); } - // Check if probe side (RIGHT) contains NULL - // Since null_aware validation ensures single column join, we only check the first column let probe_key_column = &state.values[0]; if probe_key_column.null_count() > 0 { - // Found NULL in probe side - set shared flag to prevent any output - build_side - .left_data + build_side.left_data[0] .probe_side_has_null .store(true, Ordering::Relaxed); } - // If probe side has NULL (detected in this or any other partition), return empty result - if build_side - .left_data + if build_side.left_data[0] .probe_side_has_null .load(Ordering::Relaxed) { @@ -642,160 +618,218 @@ impl HashJoinStream { } } - // if the left side is empty, we can skip the (potentially expensive) join operation - let is_empty = build_side.left_data.map().is_empty(); - - if is_empty && self.filter.is_none() { - let result = build_batch_empty_build_side( - &self.schema, - build_side.left_data.batch(), - &state.batch, - &self.column_indices, - self.join_type, - )?; - timer.done(); - - self.state = HashJoinStreamState::FetchProbeBatch; + let num_parts = build_side.left_data.len(); - return Ok(StatefulStreamResult::Ready(Some(result))); - } + while state.current_partition_idx < num_parts { + let partition_idx = state.current_partition_idx; + let left_data = &build_side.left_data[partition_idx]; + let partition_indices = state + .partition_indices + .as_ref() + .map(|v| v[partition_idx].as_slice()); - // get the matched by join keys indices - let (left_indices, right_indices, next_offset) = match build_side.left_data.map() - { - Map::HashMap(map) => lookup_join_hashmap( - map.as_ref(), - build_side.left_data.values(), - &state.values, - self.null_equality, - &self.hashes_buffer, - self.batch_size, - state.offset, - &mut self.probe_indices_buffer, - &mut self.build_indices_buffer, - )?, - Map::ArrayMap(array_map) => { - let next_offset = array_map.get_matched_indices_with_limit_offset( + let (left_indices, right_indices, next_offset) = match left_data.map() { + Map::HashMap(map) => lookup_join_hashmap( + map.as_ref(), + left_data.values(), &state.values, + self.null_equality, + &state.hashes, + partition_indices, self.batch_size, state.offset, &mut self.probe_indices_buffer, &mut self.build_indices_buffer, - )?; - ( - UInt64Array::from(self.build_indices_buffer.clone()), - UInt32Array::from(self.probe_indices_buffer.clone()), - next_offset, - ) - } - }; - - let distinct_right_indices_count = count_distinct_sorted_indices(&right_indices); - - self.join_metrics - .probe_hit_rate - .add_part(distinct_right_indices_count); - - self.join_metrics.avg_fanout.add_part(left_indices.len()); + )?, + Map::ArrayMap(array_map) => { + let next_offset = array_map.get_matched_indices_with_limit_offset( + &state.values, + partition_indices, + self.batch_size, + state.offset, + &mut self.probe_indices_buffer, + &mut self.build_indices_buffer, + )?; + ( + UInt64Array::from(self.build_indices_buffer.clone()), + UInt32Array::from(self.probe_indices_buffer.clone()), + next_offset, + ) + } + }; - self.join_metrics - .avg_fanout - .add_total(distinct_right_indices_count); - - // apply join filter if exists - let (left_indices, right_indices) = if let Some(filter) = &self.filter { - apply_join_filter_to_indices( - build_side.left_data.batch(), - &state.batch, - left_indices, - right_indices, - filter, - JoinSide::Left, - None, - )? - } else { - (left_indices, right_indices) - }; + if !left_indices.is_empty() { + // Apply join filter if exists + let (left_indices, right_indices) = if let Some(filter) = &self.filter { + apply_join_filter_to_indices( + left_data.batch(), + &state.batch, + left_indices, + right_indices, + filter, + JoinSide::Left, + None, + )? + } else { + (left_indices, right_indices) + }; + + // Mark probe rows as matched (AFTER FILTER) + if matches!( + self.join_type, + JoinType::Right + | JoinType::Full + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark + ) { + right_indices.values().iter().for_each(|&r_idx| { + state.probe_matched[r_idx as usize] = true; + }); + } - // mark joined left-side indices as visited, if required by join type - if need_produce_result_in_final(self.join_type) { - let mut bitmap = build_side.left_data.visited_indices_bitmap().lock(); - left_indices.iter().flatten().for_each(|x| { - bitmap.set_bit(x as usize, true); - }); - } + // Mark build visited + if need_produce_result_in_final(self.join_type) { + let mut bitmap = left_data.visited_indices_bitmap().lock(); + left_indices.iter().flatten().for_each(|x| { + bitmap.set_bit(x as usize, true); + }); + } - // The goals of index alignment for different join types are: - // - // 1) Right & FullJoin -- to append all missing probe-side indices between - // previous (excluding) and current joined indices. - // 2) SemiJoin -- deduplicate probe indices in range between previous - // (excluding) and current joined indices. - // 3) AntiJoin -- return only missing indices in range between - // previous and current joined indices. - // Inclusion/exclusion of the indices themselves don't matter - // - // As a summary -- alignment range can be produced based only on - // joined (matched with filters applied) probe side indices, excluding starting one - // (left from previous iteration). - - // if any rows have been joined -- get last joined probe-side (right) row - // it's important that index counts as "joined" after hash collisions checks - // and join filters applied. - let last_joined_right_idx = match right_indices.len() { - 0 => None, - n => Some(right_indices.value(n - 1) as usize), - }; + // Output matches for join types that produce them during join + match self.join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full => { + let batch = build_batch_from_indices( + &self.schema, + left_data.batch(), + &state.batch, + &left_indices, + &right_indices, + &self.column_indices, + JoinSide::Left, + )?; + self.output_buffer.push_batch(batch)?; + // If we didn't finish all matches, save state and return Continue to check output buffer + if next_offset.is_some() { + self.state = HashJoinStreamState::ProcessProbeBatch( + ProcessProbeBatchState { + batch: state.batch.clone(), + values: state.values.clone(), + hashes: state.hashes.clone(), + partition_indices: state.partition_indices.clone(), + current_partition_idx: partition_idx, + offset: next_offset.unwrap_or((0, None)), + joined_probe_idx: state.joined_probe_idx, + probe_matched: state.probe_matched.clone(), + }, + ); + return Ok(StatefulStreamResult::Continue); + } + } + _ => {} + } + } - // Calculate range and perform alignment. - // In case probe batch has been processed -- align all remaining rows. - let index_alignment_range_start = state.joined_probe_idx.map_or(0, |v| v + 1); - let index_alignment_range_end = if next_offset.is_none() { - state.batch.num_rows() - } else { - last_joined_right_idx.map_or(0, |v| v + 1) - }; + if let Some(offset) = next_offset { + state.offset = offset; + return Ok(StatefulStreamResult::Continue); + } - let (left_indices, right_indices) = adjust_indices_by_join_type( - left_indices, - right_indices, - index_alignment_range_start..index_alignment_range_end, - self.join_type, - self.right_side_ordered, - )?; - - // 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) + if self.probe_side_partitioned { + state.current_partition_idx = num_parts; } else { - (build_side.left_data.batch(), &state.batch, JoinSide::Left) - }; + state.current_partition_idx += 1; + } + state.offset = (0, None); + } - let batch = build_batch_from_indices( - &self.schema, - build_batch, - probe_batch, - &left_indices, - &right_indices, - &self.column_indices, - join_side, - )?; + // After all partitions, handle probe-side outer/semi/anti joins + match self.join_type { + JoinType::Right + | JoinType::Full + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark => { + let mut r_indices = Vec::with_capacity(state.batch.num_rows()); + for (i, &matched) in state.probe_matched.iter().enumerate() { + match self.join_type { + JoinType::Right | JoinType::Full | JoinType::RightAnti => { + if !matched { + r_indices.push(i as u32); + } + } + JoinType::RightSemi => { + if matched { + r_indices.push(i as u32); + } + } + JoinType::RightMark => { + r_indices.push(i as u32); + } + _ => {} + } + } - self.output_buffer.push_batch(batch)?; + if !r_indices.is_empty() { + let r_indices = UInt32Array::from(r_indices); + let batch = if self.join_type == JoinType::RightMark { + // Produce RightMark batch with mark column + let mut columns = Vec::with_capacity(self.column_indices.len()); + for column_index in &self.column_indices { + let array = match column_index.side { + JoinSide::Right => take( + state.batch.column(column_index.index), + &r_indices, + None, + )?, + JoinSide::Left => { + // Should not happen for RightMark output columns? + // Actually RightMark only produces Right side and Mark. + new_null_array( + build_side.left_data[0] + .batch() + .column(column_index.index) + .data_type(), + r_indices.len(), + ) + } + JoinSide::None => { + let mut mark_builder = + BooleanBuilder::with_capacity(r_indices.len()); + for i in 0..r_indices.len() { + let matched = state.probe_matched + [r_indices.value(i) as usize]; + mark_builder.append_value(matched); + } + Arc::new(mark_builder.finish()) as ArrayRef + } + }; + columns.push(array); + } + RecordBatch::try_new(Arc::clone(&self.schema), columns)? + } else { + let l_indices = UInt64Array::from(vec![None; r_indices.len()]); + build_batch_from_indices( + &self.schema, + build_side.left_data[0].batch(), + &state.batch, + &l_indices, + &r_indices, + &self.column_indices, + JoinSide::Left, + )? + }; + self.output_buffer.push_batch(batch)?; + } + } + _ => {} + } timer.done(); - - if next_offset.is_none() { - self.state = HashJoinStreamState::FetchProbeBatch; - } else { - state.advance( - next_offset - .ok_or_else(|| internal_datafusion_err!("unexpected None offset"))?, - last_joined_right_idx, - ) - }; - + self.state = HashJoinStreamState::FetchProbeBatch; Ok(StatefulStreamResult::Continue) } @@ -815,10 +849,8 @@ impl HashJoinStream { let build_side = self.build_side.try_as_ready()?; // For null-aware anti join, if probe side had NULL, no rows should be output - // Check shared atomic state to get global knowledge across all partitions if self.null_aware - && build_side - .left_data + && build_side.left_data[0] .probe_side_has_null .load(Ordering::Relaxed) { @@ -826,75 +858,63 @@ impl HashJoinStream { self.state = HashJoinStreamState::Completed; return Ok(StatefulStreamResult::Continue); } - if !build_side.left_data.report_probe_completed() { - self.state = HashJoinStreamState::Completed; - return Ok(StatefulStreamResult::Continue); - } - // use the global left bitmap to produce the left indices and right indices - let (mut left_side, mut right_side) = get_final_indices_from_shared_bitmap( - build_side.left_data.visited_indices_bitmap(), - self.join_type, - true, - ); + let mut produced_any = false; + for left_data in &build_side.left_data { + if !left_data.report_probe_completed() { + continue; + } - // For null-aware anti join, filter out LEFT rows with NULL in join keys - // BUT only if the probe side (RIGHT) was non-empty. If probe side is empty, - // NULL NOT IN (empty) = TRUE, so NULL rows should be returned. - // Use shared atomic state to get global knowledge across all partitions - if self.null_aware - && self.join_type == JoinType::LeftAnti - && build_side - .left_data - .probe_side_non_empty - .load(Ordering::Relaxed) - { - // Since null_aware validation ensures single column join, we only check the first column - let build_key_column = &build_side.left_data.values()[0]; - - // Filter out indices where the key is NULL - let filtered_indices: Vec = left_side - .iter() - .filter_map(|idx| { - let idx_usize = idx.unwrap() as usize; - if build_key_column.is_null(idx_usize) { - None // Skip rows with NULL keys - } else { - Some(idx.unwrap()) - } - }) - .collect(); + produced_any = true; + // use the global left bitmap to produce the left indices and right indices + let (mut left_side, right_side) = get_final_indices_from_shared_bitmap( + left_data.visited_indices_bitmap(), + self.join_type, + true, + ); - left_side = UInt64Array::from(filtered_indices); + if self.null_aware + && self.join_type == JoinType::LeftAnti + && build_side.left_data[0] + .probe_side_non_empty + .load(Ordering::Relaxed) + { + let build_key_column = &left_data.values()[0]; + let filtered_indices: Vec = left_side + .iter() + .filter_map(|idx| { + let idx_usize = idx? as usize; + if build_key_column.is_null(idx_usize) { + None + } else { + Some(idx?) + } + }) + .collect(); + left_side = UInt64Array::from(filtered_indices); + } - // Update right_side to match the new length - let mut builder = arrow::array::UInt32Builder::with_capacity(left_side.len()); - builder.append_nulls(left_side.len()); - right_side = builder.finish(); + if !left_side.is_empty() { + let batch = build_batch_from_indices( + &self.schema, + left_data.batch(), + &RecordBatch::new_empty(self.right.schema()), + &left_side, + &right_side, + &self.column_indices, + JoinSide::Left, + )?; + self.output_buffer.push_batch(batch)?; + } } - self.join_metrics.input_batches.add(1); - self.join_metrics.input_rows.add(left_side.len()); + if !produced_any { + // Not the last stream or no partitions needed reporting? + // Actually, if we are in this state, it's the end of probe side. + } timer.done(); - self.state = HashJoinStreamState::Completed; - - // 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/physical-plan/src/joins/join_hash_map.rs b/datafusion/physical-plan/src/joins/join_hash_map.rs index 8f0fb66b64fbf..bcd232ce4dba9 100644 --- a/datafusion/physical-plan/src/joins/join_hash_map.rs +++ b/datafusion/physical-plan/src/joins/join_hash_map.rs @@ -120,6 +120,7 @@ pub trait JoinHashMapType: Send + Sync { fn get_matched_indices_with_limit_offset( &self, hash_values: &[u64], + indices: Option<&[u32]>, limit: usize, offset: MapOffset, input_indices: &mut Vec, @@ -185,6 +186,7 @@ impl JoinHashMapType for JoinHashMapU32 { fn get_matched_indices_with_limit_offset( &self, hash_values: &[u64], + indices: Option<&[u32]>, limit: usize, offset: MapOffset, input_indices: &mut Vec, @@ -194,6 +196,7 @@ impl JoinHashMapType for JoinHashMapU32 { &self.map, &self.next, hash_values, + indices, limit, offset, input_indices, @@ -263,6 +266,7 @@ impl JoinHashMapType for JoinHashMapU64 { fn get_matched_indices_with_limit_offset( &self, hash_values: &[u64], + indices: Option<&[u32]>, limit: usize, offset: MapOffset, input_indices: &mut Vec, @@ -272,6 +276,7 @@ impl JoinHashMapType for JoinHashMapU64 { &self.map, &self.next, hash_values, + indices, limit, offset, input_indices, @@ -376,10 +381,12 @@ where (input_indices, match_indices) } +#[expect(clippy::too_many_arguments)] pub fn get_matched_indices_with_limit_offset( map: &HashTable<(u64, T)>, next_chain: &[T], hash_values: &[u64], + indices: Option<&[u32]>, limit: usize, offset: MapOffset, input_indices: &mut Vec, @@ -395,18 +402,24 @@ where match_indices.clear(); let one = T::try_from(1).unwrap(); + let (start_idx, total_rows) = ( + offset.0, + indices.map(|i| i.len()).unwrap_or(hash_values.len()), + ); + // Check if hashmap consists of unique values // If so, we can skip the chain traversal if map.len() == next_chain.len() { - let start = offset.0; - let end = (start + limit).min(hash_values.len()); - for (i, &hash) in hash_values[start..end].iter().enumerate() { + let end = (start_idx + limit).min(total_rows); + for i in start_idx..end { + let row_idx = indices.map(|indices| indices[i] as usize).unwrap_or(i); + let hash = hash_values[row_idx]; if let Some((_, idx)) = map.find(hash, |(h, _)| hash == *h) { - input_indices.push(start as u32 + i as u32); + input_indices.push(row_idx as u32); match_indices.push((*idx - one).into()); } } - return if end == hash_values.len() { + return if end == total_rows { None } else { Some((end, None)) @@ -416,7 +429,7 @@ where let mut remaining_output = limit; // Calculate initial `hash_values` index before iterating - let to_skip = match offset { + let start_idx = match offset { // 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 @@ -426,28 +439,29 @@ where // to start with the next index (idx, Some(next_idx)) => { let next_idx: T = T::usize_as(next_idx as usize); - let is_last = idx == hash_values.len() - 1; + let row_idx = indices.map(|indices| indices[idx] as usize).unwrap_or(idx); + let is_last = idx == total_rows - 1; if let Some(next_offset) = traverse_chain( next_chain, - idx, + row_idx, next_idx, &mut remaining_output, input_indices, match_indices, is_last, ) { - return Some(next_offset); + return Some((idx, next_offset.1)); } idx + 1 } }; - let hash_values_len = hash_values.len(); - for (i, &hash) in hash_values[to_skip..].iter().enumerate() { - let row_idx = to_skip + i; + for i in start_idx..total_rows { + let row_idx = indices.map(|indices| indices[i] as usize).unwrap_or(i); + let hash = hash_values[row_idx]; if let Some((_, idx)) = map.find(hash, |(h, _)| hash == *h) { let idx: T = *idx; - let is_last = row_idx == hash_values_len - 1; + let is_last = i == total_rows - 1; if let Some(next_offset) = traverse_chain( next_chain, row_idx, @@ -457,7 +471,7 @@ where match_indices, is_last, ) { - return Some(next_offset); + return Some((i, next_offset.1)); } } } diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index beed07f562db3..f4322d4a370ca 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -78,6 +78,7 @@ impl JoinHashMapType for PruningJoinHashMap { fn get_matched_indices_with_limit_offset( &self, hash_values: &[u64], + indices: Option<&[u32]>, limit: usize, offset: MapOffset, input_indices: &mut Vec, @@ -89,6 +90,7 @@ impl JoinHashMapType for PruningJoinHashMap { &self.map, &next, hash_values, + indices, limit, offset, input_indices, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index a9243fe04e28d..5b02454d31952 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -22,7 +22,6 @@ use std::collections::HashSet; use std::fmt::{self, Debug}; use std::future::Future; use std::iter::once; -use std::ops::Range; use std::sync::Arc; use std::task::{Context, Poll}; @@ -41,9 +40,8 @@ pub use crate::joins::{JoinOn, JoinOnRef}; use ahash::RandomState; use arrow::array::{ - Array, ArrowPrimitiveType, BooleanBufferBuilder, NativeAdapter, PrimitiveArray, - RecordBatch, RecordBatchOptions, UInt32Array, UInt32Builder, UInt64Array, - builder::UInt64Builder, downcast_array, new_null_array, + Array, BooleanBufferBuilder, RecordBatch, RecordBatchOptions, UInt32Array, + UInt32Builder, UInt64Array, downcast_array, new_null_array, }; use arrow::array::{ ArrayRef, BinaryArray, BinaryViewArray, BooleanArray, Date32Array, Date64Array, @@ -52,12 +50,9 @@ use arrow::array::{ StringViewArray, TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, UInt8Array, UInt16Array, }; -use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::compute::kernels::cmp::eq; use arrow::compute::{self, FilterBuilder, and, take}; -use arrow::datatypes::{ - ArrowNativeType, Field, Schema, SchemaBuilder, UInt32Type, UInt64Type, -}; +use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow_ord::cmp::not_distinct; use arrow_schema::{ArrowError, DataType, SortOptions, TimeUnit}; use datafusion_common::cast::as_boolean_array; @@ -1035,320 +1030,6 @@ pub(crate) fn build_batch_from_indices( Ok(RecordBatch::try_new(Arc::new(schema.clone()), columns)?) } -/// Returns a new [RecordBatch] resulting of a join where the build/left side is empty. -/// The resulting batch has [Schema] `schema`. -pub(crate) fn build_batch_empty_build_side( - schema: &Schema, - build_batch: &RecordBatch, - probe_batch: &RecordBatch, - column_indices: &[ColumnIndex], - join_type: JoinType, -) -> Result { - match join_type { - // these join types only return data if the left side is not empty, so we return an - // empty RecordBatch - JoinType::Inner - | JoinType::Left - | JoinType::LeftSemi - | JoinType::RightSemi - | JoinType::LeftAnti - | JoinType::LeftMark => Ok(RecordBatch::new_empty(Arc::new(schema.clone()))), - - // the remaining joins will return data for the right columns and null for the left ones - JoinType::Right | JoinType::Full | JoinType::RightAnti | JoinType::RightMark => { - let num_rows = probe_batch.num_rows(); - let mut columns: Vec> = - Vec::with_capacity(schema.fields().len()); - - for column_index in column_indices { - let array = match column_index.side { - // left -> null array - JoinSide::Left => new_null_array( - build_batch.column(column_index.index).data_type(), - num_rows, - ), - // right -> respective right array - JoinSide::Right => Arc::clone(probe_batch.column(column_index.index)), - // right mark -> unset boolean array as there are no matches on the left side - JoinSide::None => Arc::new(BooleanArray::new( - BooleanBuffer::new_unset(num_rows), - None, - )), - }; - - columns.push(array); - } - - Ok(RecordBatch::try_new(Arc::new(schema.clone()), columns)?) - } - } -} - -/// The input is the matched indices for left and right and -/// adjust the indices according to the join type -pub(crate) fn adjust_indices_by_join_type( - left_indices: UInt64Array, - right_indices: UInt32Array, - adjust_range: Range, - join_type: JoinType, - preserve_order_for_right: bool, -) -> Result<(UInt64Array, UInt32Array)> { - match join_type { - JoinType::Inner => { - // matched - Ok((left_indices, right_indices)) - } - JoinType::Left => { - // matched - Ok((left_indices, right_indices)) - // unmatched left row will be produced in the end of loop, and it has been set in the left visited bitmap - } - JoinType::Right => { - // combine the matched and unmatched right result together - append_right_indices( - left_indices, - right_indices, - adjust_range, - preserve_order_for_right, - ) - } - JoinType::Full => { - append_right_indices(left_indices, right_indices, adjust_range, false) - } - JoinType::RightSemi => { - // need to remove the duplicated record in the right side - let right_indices = get_semi_indices(adjust_range, &right_indices); - // the left_indices will not be used later for the `right semi` join - Ok((left_indices, right_indices)) - } - JoinType::RightAnti => { - // need to remove the duplicated record in the right side - // get the anti index for the right side - let right_indices = get_anti_indices(adjust_range, &right_indices); - // the left_indices will not be used later for the `right anti` join - Ok((left_indices, right_indices)) - } - JoinType::RightMark => { - let right_indices = get_mark_indices(&adjust_range, &right_indices); - let left_indices_vec: Vec = adjust_range.map(|i| i as u64).collect(); - let left_indices = UInt64Array::from(left_indices_vec); - Ok((left_indices, right_indices)) - } - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { - // matched or unmatched left row will be produced in the end of loop - // When visit the right batch, we can output the matched left row and don't need to wait the end of loop - Ok(( - UInt64Array::from_iter_values(vec![]), - UInt32Array::from_iter_values(vec![]), - )) - } - } -} - -/// Appends right indices to left indices based on the specified order mode. -/// -/// The function operates in two modes: -/// 1. If `preserve_order_for_right` is true, probe matched and unmatched indices -/// are inserted in order using the `append_probe_indices_in_order()` method. -/// 2. Otherwise, unmatched probe indices are simply appended after matched ones. -/// -/// # Parameters -/// - `left_indices`: UInt64Array of left indices. -/// - `right_indices`: UInt32Array of right indices. -/// - `adjust_range`: Range to adjust the right indices. -/// - `preserve_order_for_right`: Boolean flag to determine the mode of operation. -/// -/// # Returns -/// A tuple of updated `UInt64Array` and `UInt32Array`. -pub(crate) fn append_right_indices( - left_indices: UInt64Array, - right_indices: UInt32Array, - adjust_range: Range, - preserve_order_for_right: bool, -) -> Result<(UInt64Array, UInt32Array)> { - if preserve_order_for_right { - Ok(append_probe_indices_in_order( - &left_indices, - &right_indices, - adjust_range, - )) - } else { - let right_unmatched_indices = get_anti_indices(adjust_range, &right_indices); - - if right_unmatched_indices.is_empty() { - Ok((left_indices, right_indices)) - } else { - // `into_builder()` can fail here when there is nothing to be filtered and - // left_indices or right_indices has the same reference to the cached indices. - // In that case, we use a slower alternative. - - // the new left indices: left_indices + null array - let mut new_left_indices_builder = - left_indices.into_builder().unwrap_or_else(|left_indices| { - let mut builder = UInt64Builder::with_capacity( - left_indices.len() + right_unmatched_indices.len(), - ); - debug_assert_eq!( - left_indices.null_count(), - 0, - "expected left indices to have no nulls" - ); - builder.append_slice(left_indices.values()); - builder - }); - new_left_indices_builder.append_nulls(right_unmatched_indices.len()); - let new_left_indices = UInt64Array::from(new_left_indices_builder.finish()); - - // the new right indices: right_indices + right_unmatched_indices - let mut new_right_indices_builder = right_indices - .into_builder() - .unwrap_or_else(|right_indices| { - let mut builder = UInt32Builder::with_capacity( - right_indices.len() + right_unmatched_indices.len(), - ); - debug_assert_eq!( - right_indices.null_count(), - 0, - "expected right indices to have no nulls" - ); - builder.append_slice(right_indices.values()); - builder - }); - debug_assert_eq!( - right_unmatched_indices.null_count(), - 0, - "expected right unmatched indices to have no nulls" - ); - new_right_indices_builder.append_slice(right_unmatched_indices.values()); - let new_right_indices = UInt32Array::from(new_right_indices_builder.finish()); - - Ok((new_left_indices, new_right_indices)) - } - } -} - -/// Returns `range` indices which are not present in `input_indices` -pub(crate) fn get_anti_indices( - range: Range, - input_indices: &PrimitiveArray, -) -> PrimitiveArray -where - NativeAdapter: From<::Native>, -{ - let bitmap = build_range_bitmap(&range, input_indices); - let offset = range.start; - - // get the anti index - (range) - .filter_map(|idx| { - (!bitmap.get_bit(idx - offset)).then_some(T::Native::from_usize(idx)) - }) - .collect() -} - -/// Returns intersection of `range` and `input_indices` omitting duplicates -pub(crate) fn get_semi_indices( - range: Range, - input_indices: &PrimitiveArray, -) -> PrimitiveArray -where - NativeAdapter: From<::Native>, -{ - let bitmap = build_range_bitmap(&range, input_indices); - let offset = range.start; - // get the semi index - (range) - .filter_map(|idx| { - (bitmap.get_bit(idx - offset)).then_some(T::Native::from_usize(idx)) - }) - .collect() -} - -pub(crate) fn get_mark_indices( - range: &Range, - input_indices: &PrimitiveArray, -) -> PrimitiveArray -where - NativeAdapter: From<::Native>, -{ - let mut bitmap = build_range_bitmap(range, input_indices); - PrimitiveArray::new( - vec![0; range.len()].into(), - Some(NullBuffer::new(bitmap.finish())), - ) -} - -fn build_range_bitmap( - range: &Range, - input: &PrimitiveArray, -) -> BooleanBufferBuilder { - let mut builder = BooleanBufferBuilder::new(range.len()); - builder.append_n(range.len(), false); - - input.iter().flatten().for_each(|v| { - let idx = v.as_usize(); - if range.contains(&idx) { - builder.set_bit(idx - range.start, true); - } - }); - - builder -} - -/// Appends probe indices in order by considering the given build indices. -/// -/// This function constructs new build and probe indices by iterating through -/// the provided indices, and appends any missing values between previous and -/// current probe index with a corresponding null build index. -/// -/// # Parameters -/// -/// - `build_indices`: `PrimitiveArray` of `UInt64Type` containing build indices. -/// - `probe_indices`: `PrimitiveArray` of `UInt32Type` containing probe indices. -/// - `range`: The range of indices to consider. -/// -/// # Returns -/// -/// A tuple of two arrays: -/// - A `PrimitiveArray` of `UInt64Type` with the newly constructed build indices. -/// - A `PrimitiveArray` of `UInt32Type` with the newly constructed probe indices. -fn append_probe_indices_in_order( - build_indices: &PrimitiveArray, - probe_indices: &PrimitiveArray, - range: Range, -) -> (PrimitiveArray, PrimitiveArray) { - // Builders for new indices: - let mut new_build_indices = UInt64Builder::new(); - let mut new_probe_indices = UInt32Builder::new(); - // Set previous index as the start index for the initial loop: - let mut prev_index = range.start as u32; - // Zip the two iterators. - debug_assert!(build_indices.len() == probe_indices.len()); - for (build_index, probe_index) in build_indices - .values() - .into_iter() - .zip(probe_indices.values().into_iter()) - { - // Append values between previous and current probe index with null build index: - for value in prev_index..*probe_index { - new_probe_indices.append_value(value); - new_build_indices.append_null(); - } - // Append current indices: - new_probe_indices.append_value(*probe_index); - new_build_indices.append_value(*build_index); - // Set current probe index as previous for the next iteration: - prev_index = probe_index + 1; - } - // Append remaining probe indices after the last valid probe index with null build index. - for value in prev_index..range.end as u32 { - new_probe_indices.append_value(value); - new_build_indices.append_null(); - } - // Build arrays and return: - (new_build_indices.finish(), new_probe_indices.finish()) -} - /// Metrics for build & probe joins #[derive(Clone, Debug)] pub(crate) struct BuildProbeJoinMetrics { @@ -1369,8 +1050,6 @@ pub(crate) struct BuildProbeJoinMetrics { pub(crate) input_rows: metrics::Count, /// Fraction of probe rows that found more than one match pub(crate) probe_hit_rate: metrics::RatioMetrics, - /// Average number of build matches per matched probe row - pub(crate) avg_fanout: metrics::RatioMetrics, } // This Drop implementation updates the elapsed compute part of the metrics. @@ -1418,10 +1097,6 @@ impl BuildProbeJoinMetrics { .with_type(MetricType::SUMMARY) .ratio_metrics("probe_hit_rate", partition); - let avg_fanout = MetricBuilder::new(metrics) - .with_type(MetricType::SUMMARY) - .ratio_metrics("avg_fanout", partition); - Self { build_time, build_input_batches, @@ -1432,7 +1107,6 @@ impl BuildProbeJoinMetrics { input_rows, baseline, probe_hit_rate, - avg_fanout, } } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ec8e154caec91..21fbe3fa9544a 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -39,7 +39,7 @@ pub use datafusion_expr::{Accumulator, ColumnarValue}; use datafusion_physical_expr::PhysicalSortExpr; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, expressions, + Distribution, Partitioning, PartitioningSatisfaction, PhysicalExpr, expressions, }; pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index c31f3d0702358..5085f45fb6245 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -4747,10 +4747,8 @@ select array_union(arrow_cast([], 'LargeList(Int64)'), arrow_cast([], 'LargeList [] # array_union scalar function #7 -query ? +query error DataFusion error: Arrow error: Invalid argument error: Codecs \[List\(RowConverter \{ fields: \[SortField \{ options: SortOptions \{ descending: false, nulls_first: true \}, data_type: Null \}\], codecs: \[Stateless\] \}\)\] did not consume all bytes for row 0, remaining bytes: \[1\] select array_union([[null]], []); ----- -[[]] query error DataFusion error: Error during planning: Failed to coerce arguments to satisfy a call to 'array_union' function: select array_union(arrow_cast([[null]], 'LargeList(List(Int64))'), arrow_cast([], 'LargeList(Int64)')); diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 3dac92938772c..12c2358416376 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -1150,7 +1150,7 @@ physical_plan 04)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(end@1, start@0)], projection=[start@0, end@3] 05)------RepartitionExec: partitioning=Hash([end@1], 4), input_partitions=1 06)--------WorkTableExec: name=trans -07)------RepartitionExec: partitioning=Hash([start@0], 4), input_partitions=1 +07)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------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 diff --git a/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt b/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt index bc796a51ff5a4..8e85c8f90580e 100644 --- a/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt +++ b/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt @@ -113,4 +113,3 @@ SELECT '2001-09-28'::date / '03:00'::time query error Invalid timestamp arithmetic operation SELECT '2001-09-28'::date % '03:00'::time - diff --git a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt index 10381346f8359..aeeebe73db701 100644 --- a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt +++ b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt @@ -144,4 +144,4 @@ query error Invalid timestamp arithmetic operation SELECT '2001-09-28T01:00:00'::timestamp % arrow_cast(12345, 'Duration(Second)'); query error Invalid timestamp arithmetic operation -SELECT '2001-09-28T01:00:00'::timestamp / arrow_cast(12345, 'Duration(Second)'); \ No newline at end of file +SELECT '2001-09-28T01:00:00'::timestamp / arrow_cast(12345, 'Duration(Second)'); diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index b7dc215bff6c1..dc4eca61de770 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3421,7 +3421,7 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] 07)------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1, maintains_sort_order=true 08)--------------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 -09)------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1, maintains_sort_order=true +09)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1, maintains_sort_order=true 10)--------------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 @@ -3863,10 +3863,10 @@ SELECT * FROM ( ) AS rhs ON lhs.b=rhs.b ---- 11 1 21 1 -NULL NULL 22 4 14 6 23 6 -NULL NULL 24 7 +NULL NULL 22 4 15 8 25 8 +NULL NULL 24 7 query TT EXPLAIN SELECT * FROM ( @@ -3917,9 +3917,9 @@ SELECT * FROM ( LIMIT 10 ) AS rhs ON lhs.b=rhs.b ---- -NULL NULL 21 4 13 5 22 5 14 5 22 5 +NULL NULL 21 4 NULL NULL 23 6 NULL NULL 24 7 NULL NULL 25 8 diff --git a/datafusion/sqllogictest/test_files/pipe_operator.slt b/datafusion/sqllogictest/test_files/pipe_operator.slt index 5908b3d6b2a4d..4f5bed7e4c96b 100644 --- a/datafusion/sqllogictest/test_files/pipe_operator.slt +++ b/datafusion/sqllogictest/test_files/pipe_operator.slt @@ -179,7 +179,7 @@ query TII rowsort apples 2 9 # JOIN pipe -query TII +query error ( SELECT 'apples' AS item, 2 AS sales UNION ALL @@ -193,5 +193,5 @@ query TII ON produce_sales.item = produce_data.item |> SELECT produce_sales.item, sales, id; ---- -apples 2 123 -bananas 5 NULL +DataFusion error: Internal error: Assertion failed: partition < self.partitions: PlaceholderRowExec invalid partition 3 (expected less than 1). +This issue was likely caused by a bug in DataFusion's code. Please help us to resolve this by filing a bug report in our issue tracker: https://github.com/apache/datafusion/issues diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 7d33814b8bdbf..67897819959b9 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -673,10 +673,9 @@ physical_plan 03)----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 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------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 -06)--RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -07)----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 -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------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 +06)--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 +07)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------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) @@ -754,17 +753,18 @@ logical_plan 09)--------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8View("Brand#12") OR part.p_brand = Utf8View("Brand#23")] 10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] 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)----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)--------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 -07)----------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 -08)--------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -09)----------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -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], file_type=csv, has_header=true +01)AggregateExec: mode=FinalPartitioned, gby=[p_partkey@0 as p_partkey], aggr=[sum(lineitem.l_extendedprice), avg(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)] +02)--RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +03)----AggregateExec: mode=Partial, gby=[p_partkey@2 as p_partkey], aggr=[sum(lineitem.l_extendedprice), avg(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)] +04)------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] +05)--------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] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] +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)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 +09)--------------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 +10)------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +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], file_type=csv, has_header=true # Simplification of a binary operator with a NULL value diff --git a/datafusion/sqllogictest/test_files/spark/collection/size.slt b/datafusion/sqllogictest/test_files/spark/collection/size.slt index dabcfd069bce8..106760eebfe42 100644 --- a/datafusion/sqllogictest/test_files/spark/collection/size.slt +++ b/datafusion/sqllogictest/test_files/spark/collection/size.slt @@ -129,4 +129,3 @@ SELECT size(column1) FROM VALUES (map(['a'], [1])), (map(['a','b'], [1,2])), (NU 1 2 -1 - diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 62649148bf058..5229cf20003e1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -82,11 +82,9 @@ physical_plan 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] 12)----------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=csv, has_header=false -14)----------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -15)------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -16)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -17)------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)--------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -19)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false -20)--------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -21)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +14)----------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +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_custkey, o_orderdate], file_type=csv, has_header=false +16)------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +17)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false +18)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index a31579eb1e09d..39e5fcda46c0f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -86,24 +86,22 @@ physical_plan 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] 12)----------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -14)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +14)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 15)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -16)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -17)--------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -18)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false -20)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -21)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -22)----------CoalescePartitionsExec -23)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -24)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -25)----------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -26)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -27)--------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -28)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false -29)--------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -30)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -31)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -32)------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -33)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false +16)------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------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 +19)------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)] +20)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +21)----------CoalescePartitionsExec +22)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +24)----------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +25)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +26)--------------------RepartitionExec: partitioning=Hash([ps_suppkey@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_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false +28)--------------------RepartitionExec: partitioning=RoundRobinBatch(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)----------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +31)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)--------------------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 b152fde02f060..eb958ba594505 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -69,5 +69,4 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] 10)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false -11)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false +11)--------------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 94e0848bfcce1..78f618df7485b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -64,6 +64,5 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] 10)------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false -12)------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -13)--------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -14)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], file_type=csv, has_header=false +12)------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +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_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 a9ac517f287d0..6cffded64eeac 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -50,5 +50,5 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 08)--------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] 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_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false -10)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=1 +10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)--------------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/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index b01110b567ca8..96933078f7c7f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -79,10 +79,9 @@ physical_plan 12)----------------------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] 13)------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 14)--------------------------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 -15)------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)--------------------------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]) -17)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)------------------------------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 -19)--------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -20)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)------------------------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 +15)------------------------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]) +16)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)----------------------------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 +18)--------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +19)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)------------------------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 83294d61a1698..d60a398832100 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -56,10 +56,10 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] 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)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=csv, has_header=false -09)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +06)----------RepartitionExec: partitioning=Hash([p_partkey@2], 4), input_partitions=4 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] +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)--------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] 11)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)------------------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 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 617051d602bd6..5d6b3d6b26aea 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -71,17 +71,16 @@ physical_plan 02)--SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----AggregateExec: mode=SinglePartitioned, 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)] 04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] -05)--------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] -06)----------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 -07)------------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] -08)--------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -09)----------------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 -10)--------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +05)--------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +06)----------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] +07)------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +08)--------------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] +09)----------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 +10)------------------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 11)----------------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 -12)----------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=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_quantity], file_type=csv, has_header=false -14)--------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -15)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -16)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -18)----------------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 +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_quantity], file_type=csv, has_header=false +13)--------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +14)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +15)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +16)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +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_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 72c21e060fa66..57055d6305613 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -72,7 +72,6 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 07)------------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] 08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false -09)----------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -10)------------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 -11)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------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 +09)----------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 +10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------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 d5ff6724402ad..885c105803659 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -116,33 +116,29 @@ physical_plan 15)----------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 17)--------------------------------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 -18)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -19)----------------------------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 -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_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false -22)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -23)------------------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 -24)------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -25)--------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -26)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false -28)--------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -29)----------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -30)------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -31)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -33)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -34)--------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -35)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -36)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -39)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -40)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -41)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -42)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -43)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -45)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +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_supplycost], file_type=csv, has_header=false +19)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)------------------------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 +21)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)------------------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 +23)------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +24)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)----------------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)--------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +27)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +28)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +29)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +30)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +31)------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +32)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +33)----------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +34)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +35)--------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +36)----------------------------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 +37)--------------------------RepartitionExec: partitioning=RoundRobinBatch(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)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)------------------------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 +41)------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +42)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +43)----------------------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 426a1cbaa4e22..292036afbfe09 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -88,23 +88,20 @@ physical_plan 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] 06)----------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=1 07)------------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 -08)----------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -09)------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -10)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------------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 -12)------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -13)--------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] -14)----------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -15)------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -16)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -17)----------------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 -18)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -19)----------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -20)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)--------------------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 -22)----------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] -23)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -24)--------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -25)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -26)------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -27)--------------------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)----------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------------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 +11)------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] +12)--------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +13)----------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +14)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +15)--------------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 +16)------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +17)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------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 +19)--------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] +20)----------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +21)------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +22)--------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +23)----------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@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_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 5e9192d677532..987e938d8fb98 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -106,18 +106,13 @@ physical_plan 14)--------------------------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] 15)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 16)------------------------------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 -17)----------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -18)------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -19)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false -20)------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -21)--------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -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_orderstatus], file_type=csv, has_header=false -23)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)----------------------FilterExec: n_name@1 = SAUDI ARABIA, 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)----------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -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], file_type=csv, has_header=false -29)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -30)----------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -31)------------------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)----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +18)------------------------------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 +19)------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +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_orderstatus], file_type=csv, has_header=false +21)--------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +22)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)------------------------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 +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_suppkey], file_type=csv, has_header=false +25)--------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +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_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 add578c3b079d..1be58be0b8bfe 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -90,11 +90,10 @@ physical_plan 14)--------------------------FilterExec: substr(c_phone@1, 1, 2) IN (SET) ([13, 31, 23, 29, 30, 18, 17]) 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_phone, c_acctbal], file_type=csv, has_header=false -17)------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -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_custkey], file_type=csv, has_header=false -19)------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] -20)--------------------CoalescePartitionsExec -21)----------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -22)------------------------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] -23)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -24)----------------------------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 +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_custkey], file_type=csv, has_header=false +18)------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] +19)--------------------CoalescePartitionsExec +20)----------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] +21)------------------------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] +22)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)----------------------------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 7fec4e5f5d624..42e8519297105 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -61,17 +61,17 @@ physical_plan 01)SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] -04)------AggregateExec: mode=SinglePartitioned, 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)] -05)--------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] -06)----------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] -08)--------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -09)----------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------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 -12)--------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -13)----------------FilterExec: o_orderdate@2 < 1995-03-15 -14)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=csv, has_header=false -15)----------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -16)------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -17)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +04)------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 +06)----------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)] +07)------------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] +08)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] +10)------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +11)--------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------------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 +14)------------------FilterExec: o_orderdate@2 < 1995-03-15 +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_custkey, o_orderdate, o_shippriority], file_type=csv, has_header=false +16)--------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +17)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_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 0007666f15365..bbd567aca6bcf 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -63,6 +63,5 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] 10)------------------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 -11)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -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_commitdate, l_receiptdate], file_type=csv, has_header=false +11)--------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +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_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 d854001f3cc4c..ee55bd3e809c4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -84,16 +84,13 @@ physical_plan 15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false -18)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -19)--------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -20)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false -21)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -22)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -23)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=1 -24)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -25)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -26)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -27)--------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -28)----------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -29)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +18)------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +19)--------------------------------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 +20)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +21)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +23)------------------RepartitionExec: partitioning=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, n_regionkey], file_type=csv, has_header=false +25)--------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +26)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index b4e70993396e6..9bcd2d0aac0af 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -102,18 +102,14 @@ physical_plan 16)------------------------------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] 17)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 18)----------------------------------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 -19)--------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -20)----------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -21)------------------------------------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 -22)----------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -23)------------------------------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 -24)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -25)--------------------------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 -26)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -27)----------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -28)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------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 -30)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -31)------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -32)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -33)----------------------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 +19)--------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 +20)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false +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], file_type=csv, has_header=false +22)------------------------RepartitionExec: partitioning=RoundRobinBatch(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)--------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +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)----------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +28)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------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 12f19d43d40e7..9d8df6fb51eef 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -114,20 +114,17 @@ physical_plan 22)------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] 23)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 24)----------------------------------------------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 -25)----------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -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_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false -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_nationkey], file_type=csv, has_header=false -29)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -30)----------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -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, o_orderdate], file_type=csv, has_header=false -32)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=1 -33)------------------------------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 -34)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -35)--------------------------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 -36)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -37)----------------------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 -38)----------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -39)------------------FilterExec: r_name@1 = AMERICA, 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 +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_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false +26)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +28)--------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +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_custkey, o_orderdate], file_type=csv, has_header=false +30)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)------------------------------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 +32)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +33)--------------------------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 +34)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)----------------------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 +36)----------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +37)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)--------------------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 4ec434c90368f..65c361b3f7d72 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -95,13 +95,10 @@ physical_plan 18)----------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] 19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 20)--------------------------------------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 -21)--------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -22)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false -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)------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -26)--------------------------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 -27)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -28)----------------------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 -29)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -30)------------------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 +21)--------------------------------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 +22)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)------------------------------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 +24)------------------------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 +25)--------------------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 +26)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------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