From b97cd24f8ec4ae449ffd7fc3de44617390d2e234 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 26 Mar 2025 01:44:04 +0800 Subject: [PATCH 01/37] introduce selection vector repartitioning --- .../physical-plan/src/repartition/mod.rs | 80 ++++++++++++++++++- 1 file changed, 79 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c86a37697a05..a40908edc8a9 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,9 +40,13 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions}; +use arrow::array::{ + ArrayRef, BooleanArray, Int32Array, Int64Array, PrimitiveArray, RecordBatch, + RecordBatchOptions, UInt64Array, +}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; +use arrow_schema::{DataType, Field, Schema}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; use datafusion_common::HashMap; @@ -197,6 +201,12 @@ enum BatchPartitionerState { num_partitions: usize, next_idx: usize, }, + HashSelectVector { + random_state: ahash::RandomState, + exprs: Vec>, + num_partitions: usize, + hash_buffer: Vec, + }, } impl BatchPartitioner { @@ -320,6 +330,71 @@ impl BatchPartitioner { Ok((partition, batch)) }); + Box::new(it) + } + BatchPartitionerState::HashSelectVector { + random_state, + exprs, + num_partitions, + hash_buffer, + } => { + let timer = self.timer.timer(); + let arrays = exprs + .iter() + .map(|expr| expr.evaluate(&batch)?.into_array(batch.num_rows())) + .collect::>>()?; + + create_hashes(&arrays, random_state, hash_buffer)?; + + let hash_vector = hash_buffer + .iter() + .map(|hash| *hash % *num_partitions as u64) + .collect::>(); + let mut fields = batch + .schema() + .fields() + .iter() + .map(|f| Arc::clone(f)) + .collect::>(); + fields.push(Arc::new(Field::new( + "selection", + DataType::Boolean, + false, + ))); + // Finished building index-arrays for output partitions + timer.done(); + + // Borrowing partitioner timer to prevent moving `self` to closure + let partitioner_timer = &self.timer; + + let it = (0..*num_partitions).into_iter().map(move |partition| { + // Tracking time required for repartitioned batches construction + let _timer = partitioner_timer.timer(); + let select_vector = hash_vector + .iter() + .map(|&hash| hash == partition as u64) + .collect::>(); + let new_col: ArrayRef = + Arc::new(BooleanArray::from(select_vector)) as ArrayRef; + let mut columns = batch + .columns() + .iter() + .map(|c| c.clone()) + .collect::>(); + columns.push(new_col); + + let mut options = RecordBatchOptions::new(); + options = options.with_row_count(Some(batch.num_rows())); + let batch = RecordBatch::try_new_with_options( + batch.schema(), + columns, + &options, + ) + .unwrap(); + + Ok((partition, batch)) + }); + Box::new(it) } }; @@ -332,6 +407,9 @@ impl BatchPartitioner { match self.state { BatchPartitionerState::RoundRobin { num_partitions, .. } => num_partitions, BatchPartitionerState::Hash { num_partitions, .. } => num_partitions, + BatchPartitionerState::HashSelectVector { num_partitions, .. } => { + num_partitions + } } } } From c26db8518aa7198b6ab080e3cde4288fd3ec7a86 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 27 Mar 2025 01:28:06 +0800 Subject: [PATCH 02/37] finish the phsyical plan side --- datafusion/core/src/physical_planner.rs | 7 +- datafusion/physical-expr/src/partitioning.rs | 30 ++++++- .../src/enforce_distribution.rs | 5 +- .../physical-plan/src/repartition/mod.rs | 78 +++++++++++++++---- 4 files changed, 100 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index be24206c676c..ffc960c15e34 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -804,7 +804,12 @@ impl DefaultPhysicalPlanner { ) }) .collect::>>()?; - Partitioning::Hash(runtime_expr, *n) + if session_state.config_options().optimizer.prefer_hash_selection_vector_partitioning { + Partitioning::HashSelectionVector(runtime_expr, *n) + } + else { + Partitioning::Hash(runtime_expr, *n) + } } LogicalPartitioning::DistributeBy(_) => { return not_impl_err!( diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index eb7e1ea6282b..70bcf23daa7d 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -117,6 +117,13 @@ pub enum Partitioning { /// Allocate rows based on a hash of one of more expressions and the specified number of /// partitions Hash(Vec>, usize), + /// Allocate rows based on a hash of one of more expressions and the specified number + /// of partitions with a selection vecotr column. + /// + /// The column is a boolean column called `selection` that is used to filter out rows + /// that should not be included in the partition. `true` means the row should be included + /// and `false` means the row should be excluded. + HashSelectionVector(Vec>, usize), /// Unknown partitioning scheme with a known number of partitions UnknownPartitioning(usize), } @@ -133,6 +140,14 @@ impl Display for Partitioning { .join(", "); write!(f, "Hash([{phy_exprs_str}], {size})") } + Partitioning::HashSelectionVector(phy_exprs, size) => { + let phy_exprs_str = phy_exprs + .iter() + .map(|e| format!("{e}")) + .collect::>() + .join(", "); + write!(f, "HashSelectionVector([{phy_exprs_str}], {size})") + } Partitioning::UnknownPartitioning(size) => { write!(f, "UnknownPartitioning({size})") } @@ -144,7 +159,10 @@ impl Partitioning { pub fn partition_count(&self) -> usize { use Partitioning::*; match self { - RoundRobinBatch(n) | Hash(_, n) | UnknownPartitioning(n) => *n, + RoundRobinBatch(n) + | Hash(_, n) + | HashSelectionVector(_, n) + | UnknownPartitioning(n) => *n, } } @@ -165,7 +183,7 @@ impl Partitioning { // Here we do not check the partition count for hash partitioning and assumes the partition count // and hash functions in the system are the same. In future if we plan to support storage partition-wise joins, // then we need to have the partition count and hash functions validation. - Partitioning::Hash(partition_exprs, _) => { + Partitioning::Hash(partition_exprs, _) | Partitioning::HashSelectionVector(partition_exprs, _ ) => { let fast_match = physical_exprs_equal(required_exprs, partition_exprs); // If the required exprs do not match, need to leverage the eq_properties provided by the child @@ -252,14 +270,18 @@ pub enum Distribution { impl Distribution { /// Creates a `Partitioning` that satisfies this `Distribution` - pub fn create_partitioning(self, partition_count: usize) -> Partitioning { + pub fn create_partitioning(self, partition_count: usize, prefer_selection_vector: bool) -> Partitioning { match self { Distribution::UnspecifiedDistribution => { Partitioning::UnknownPartitioning(partition_count) } Distribution::SinglePartition => Partitioning::UnknownPartitioning(1), Distribution::HashPartitioned(expr) => { - Partitioning::Hash(expr, partition_count) + if prefer_selection_vector { + Partitioning::HashSelectionVector(expr, partition_count) + } else { + Partitioning::Hash(expr, partition_count) + } } } } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 700b00c19dd5..c91331927cfa 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -889,6 +889,7 @@ fn add_hash_on_top( input: DistributionContext, hash_exprs: Vec>, n_target: usize, + prefer_selection_vector: bool, ) -> Result { // Early return if hash repartition is unnecessary // `RepartitionExec: partitioning=Hash([...], 1), input_partitions=1` is unnecessary. @@ -913,7 +914,7 @@ fn add_hash_on_top( // requirements. // - Usage of order preserving variants is not desirable (per the flag // `config.optimizer.prefer_existing_sort`). - let partitioning = dist.create_partitioning(n_target); + let partitioning = dist.create_partitioning(n_target, prefer_selection_vector); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? .with_preserve_order(); @@ -1272,7 +1273,7 @@ pub fn ensure_distribution( // When inserting hash is necessary to satisfy hash requirement, insert hash repartition. if hash_necessary { child = - add_hash_on_top(child, exprs.to_vec(), target_partitions)?; + add_hash_on_top(child, exprs.to_vec(), target_partitions, config.optimizer.prefer_hash_selection_vector_partitioning)?; } } Distribution::UnspecifiedDistribution => { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index a40908edc8a9..bc07c8bb1f5b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -41,12 +41,11 @@ use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; use arrow::array::{ - ArrayRef, BooleanArray, Int32Array, Int64Array, PrimitiveArray, RecordBatch, - RecordBatchOptions, UInt64Array, + ArrayRef, BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions, }; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; -use arrow_schema::{DataType, Field, Schema}; +use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; use datafusion_common::HashMap; @@ -201,7 +200,7 @@ enum BatchPartitionerState { num_partitions: usize, next_idx: usize, }, - HashSelectVector { + HashSelectionVector { random_state: ahash::RandomState, exprs: Vec>, num_partitions: usize, @@ -209,6 +208,8 @@ enum BatchPartitionerState { }, } +pub static SELECTION_FILED_NAME: &str = "selection"; + impl BatchPartitioner { /// Create a new [`BatchPartitioner`] with the provided [`Partitioning`] /// @@ -228,6 +229,15 @@ impl BatchPartitioner { random_state: ahash::RandomState::with_seeds(0, 0, 0, 0), hash_buffer: vec![], }, + Partitioning::HashSelectionVector(exprs, num_partitions) => { + BatchPartitionerState::HashSelectionVector { + exprs, + num_partitions, + // Use fixed random hash + random_state: ahash::RandomState::with_seeds(0, 0, 0, 0), + hash_buffer: vec![], + } + } other => return not_impl_err!("Unsupported repartitioning scheme {other:?}"), }; @@ -332,7 +342,7 @@ impl BatchPartitioner { Box::new(it) } - BatchPartitionerState::HashSelectVector { + BatchPartitionerState::HashSelectionVector { random_state, exprs, num_partitions, @@ -344,6 +354,8 @@ impl BatchPartitioner { .map(|expr| expr.evaluate(&batch)?.into_array(batch.num_rows())) .collect::>>()?; + hash_buffer.clear(); + hash_buffer.resize(batch.num_rows(), 0); create_hashes(&arrays, random_state, hash_buffer)?; let hash_vector = hash_buffer @@ -354,20 +366,21 @@ impl BatchPartitioner { .schema() .fields() .iter() - .map(|f| Arc::clone(f)) + .map(Arc::clone) .collect::>(); fields.push(Arc::new(Field::new( - "selection", + SELECTION_FILED_NAME, DataType::Boolean, false, ))); + let schema = Arc::new(arrow::datatypes::Schema::new(fields)); // Finished building index-arrays for output partitions timer.done(); // Borrowing partitioner timer to prevent moving `self` to closure let partitioner_timer = &self.timer; - let it = (0..*num_partitions).into_iter().map(move |partition| { + let it = (0..*num_partitions).map(move |partition| { // Tracking time required for repartitioned batches construction let _timer = partitioner_timer.timer(); let select_vector = hash_vector @@ -379,14 +392,13 @@ impl BatchPartitioner { let mut columns = batch .columns() .iter() - .map(|c| c.clone()) + .map(Arc::clone) .collect::>(); columns.push(new_col); - let mut options = RecordBatchOptions::new(); options = options.with_row_count(Some(batch.num_rows())); let batch = RecordBatch::try_new_with_options( - batch.schema(), + Arc::clone(&schema), columns, &options, ) @@ -407,7 +419,7 @@ impl BatchPartitioner { match self.state { BatchPartitionerState::RoundRobin { num_partitions, .. } => num_partitions, BatchPartitionerState::Hash { num_partitions, .. } => num_partitions, - BatchPartitionerState::HashSelectVector { num_partitions, .. } => { + BatchPartitionerState::HashSelectionVector { num_partitions, .. } => { num_partitions } } @@ -1191,7 +1203,7 @@ mod tests { {collect, expressions::col}, }; - use arrow::array::{ArrayRef, StringArray, UInt32Array}; + use arrow::array::{ArrayRef, AsArray, StringArray, UInt32Array}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::cast::as_string_array; use datafusion_common::test_util::batches_to_sort_string; @@ -1283,6 +1295,46 @@ mod tests { Ok(()) } + #[tokio::test] + async fn many_to_many_hash_select_vector() -> Result<()> { + let schema = test_schema(); + let partition = create_vec_batches(50); + let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; + + let output_partitions = repartition( + &schema, + partitions, + Partitioning::HashSelectionVector(vec![col("c0", &schema)?], 8), + ) + .await?; + + let total_rows: usize = output_partitions + .iter() + .map(|x| { + x.iter() + .map(|x| { + x.column_by_name(SELECTION_FILED_NAME) + .unwrap() + .as_boolean() + .iter() + .filter(|x| x.unwrap_or(false)) + .count() + }) + .sum::() + }) + .sum(); + + let expected_schema = Arc::new(Schema::new(vec![ + Field::new("c0", DataType::UInt32, false), + Field::new(SELECTION_FILED_NAME, DataType::Boolean, false), + ])); + + assert_eq!(expected_schema, output_partitions[0][0].schema()); + assert_eq!(8, output_partitions.len()); + assert_eq!(total_rows, 8 * 50 * 3); + Ok(()) + } + fn test_schema() -> Arc { Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) } From 65a3031dc9289622e24403701103c249bf10cb2e Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 27 Mar 2025 01:28:26 +0800 Subject: [PATCH 03/37] add config --- datafusion/common/src/config.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index d6530931cb24..dcbd69a5bc41 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -700,6 +700,10 @@ config_namespace! { /// HashJoin can work more efficiently than SortMergeJoin but consumes more memory pub prefer_hash_join: bool, default = true + /// When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec + /// over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. + pub prefer_hash_selection_vector_partitioning: bool, default = false + /// The maximum estimated size in bytes for one input side of a HashJoin /// will be collected into a single partition pub hash_join_single_partition_threshold: usize, default = 1024 * 1024 From 271658f8bd98ce151396cc0af49352a6fb3e536e Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 27 Mar 2025 01:28:59 +0800 Subject: [PATCH 04/37] support for proto --- datafusion/proto/proto/datafusion.proto | 7 + datafusion/proto/src/generated/pbjson.rs | 128 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 12 +- .../proto/src/physical_plan/from_proto.rs | 33 +++++ .../proto/src/physical_plan/to_proto.rs | 11 ++ 5 files changed, 190 insertions(+), 1 deletion(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 4c8b6c588d94..207141a7cf64 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1225,12 +1225,18 @@ message PhysicalHashRepartition { uint64 partition_count = 2; } +message PhysicalHashSelectionVectorRepartition { + repeated PhysicalExprNode hash_expr = 1; + uint64 partition_count = 2; +} + message RepartitionExecNode{ PhysicalPlanNode input = 1; // oneof partition_method { // uint64 round_robin = 2; // PhysicalHashRepartition hash = 3; // uint64 unknown = 4; + // PhysicalHashSelectionVectorRepartition hash_selection_vector = 6; // } Partitioning partitioning = 5; } @@ -1240,6 +1246,7 @@ message Partitioning { uint64 round_robin = 1; PhysicalHashRepartition hash = 2; uint64 unknown = 3; + PhysicalHashSelectionVectorRepartition hash_selection_vector = 4; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 932422944508..0b858f4d23db 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -13403,6 +13403,9 @@ impl serde::Serialize for Partitioning { #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("unknown", ToString::to_string(&v).as_str())?; } + partitioning::PartitionMethod::HashSelectionVector(v) => { + struct_ser.serialize_field("hashSelectionVector", v)?; + } } } struct_ser.end() @@ -13419,6 +13422,8 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin", "hash", "unknown", + "hash_selection_vector", + "hashSelectionVector", ]; #[allow(clippy::enum_variant_names)] @@ -13426,6 +13431,7 @@ impl<'de> serde::Deserialize<'de> for Partitioning { RoundRobin, Hash, Unknown, + HashSelectionVector, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -13450,6 +13456,7 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin" | "round_robin" => Ok(GeneratedField::RoundRobin), "hash" => Ok(GeneratedField::Hash), "unknown" => Ok(GeneratedField::Unknown), + "hashSelectionVector" | "hash_selection_vector" => Ok(GeneratedField::HashSelectionVector), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -13491,6 +13498,13 @@ impl<'de> serde::Deserialize<'de> for Partitioning { } partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| partitioning::PartitionMethod::Unknown(x.0)); } + GeneratedField::HashSelectionVector => { + if partition_method__.is_some() { + return Err(serde::de::Error::duplicate_field("hashSelectionVector")); + } + partition_method__ = map_.next_value::<::std::option::Option<_>>()?.map(partitioning::PartitionMethod::HashSelectionVector) +; + } } } Ok(Partitioning { @@ -15060,6 +15074,120 @@ impl<'de> serde::Deserialize<'de> for PhysicalHashRepartition { deserializer.deserialize_struct("datafusion.PhysicalHashRepartition", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for PhysicalHashSelectionVectorRepartition { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.hash_expr.is_empty() { + len += 1; + } + if self.partition_count != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalHashSelectionVectorRepartition", len)?; + if !self.hash_expr.is_empty() { + struct_ser.serialize_field("hashExpr", &self.hash_expr)?; + } + if self.partition_count != 0 { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PhysicalHashSelectionVectorRepartition { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "hash_expr", + "hashExpr", + "partition_count", + "partitionCount", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + HashExpr, + PartitionCount, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "hashExpr" | "hash_expr" => Ok(GeneratedField::HashExpr), + "partitionCount" | "partition_count" => Ok(GeneratedField::PartitionCount), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PhysicalHashSelectionVectorRepartition; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.PhysicalHashSelectionVectorRepartition") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut hash_expr__ = None; + let mut partition_count__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::HashExpr => { + if hash_expr__.is_some() { + return Err(serde::de::Error::duplicate_field("hashExpr")); + } + hash_expr__ = Some(map_.next_value()?); + } + GeneratedField::PartitionCount => { + if partition_count__.is_some() { + return Err(serde::de::Error::duplicate_field("partitionCount")); + } + partition_count__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(PhysicalHashSelectionVectorRepartition { + hash_expr: hash_expr__.unwrap_or_default(), + partition_count: partition_count__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.PhysicalHashSelectionVectorRepartition", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for PhysicalInListNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index c2f4e93cef6a..4189aed9113f 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1835,6 +1835,13 @@ pub struct PhysicalHashRepartition { pub partition_count: u64, } #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PhysicalHashSelectionVectorRepartition { + #[prost(message, repeated, tag = "1")] + pub hash_expr: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "2")] + pub partition_count: u64, +} +#[derive(Clone, PartialEq, ::prost::Message)] pub struct RepartitionExecNode { #[prost(message, optional, boxed, tag = "1")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, @@ -1842,13 +1849,14 @@ pub struct RepartitionExecNode { /// uint64 round_robin = 2; /// PhysicalHashRepartition hash = 3; /// uint64 unknown = 4; + /// PhysicalHashSelectionVectorRepartition hash_selection_vector = 6; /// } #[prost(message, optional, tag = "5")] pub partitioning: ::core::option::Option, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct Partitioning { - #[prost(oneof = "partitioning::PartitionMethod", tags = "1, 2, 3")] + #[prost(oneof = "partitioning::PartitionMethod", tags = "1, 2, 3, 4")] pub partition_method: ::core::option::Option, } /// Nested message and enum types in `Partitioning`. @@ -1861,6 +1869,8 @@ pub mod partitioning { Hash(super::PhysicalHashRepartition), #[prost(uint64, tag = "3")] Unknown(u64), + #[prost(message, tag = "4")] + HashSelectionVector(super::PhysicalHashSelectionVectorRepartition), } } #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index c412dfed5d03..b2ea5aef4dd1 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -440,6 +440,31 @@ pub fn parse_protobuf_hash_partitioning( } } +pub fn parse_protobuf_hash_selection_vector_partitioning( + partitioning: Option<&protobuf::PhysicalHashSelectionVectorRepartition>, + registry: &dyn FunctionRegistry, + input_schema: &Schema, + codec: &dyn PhysicalExtensionCodec, +) -> Result> { + match partitioning { + Some(hash_part) => { + let expr = parse_physical_exprs( + &hash_part.hash_expr, + registry, + input_schema, + codec, + )?; + + Ok(Some(Partitioning::HashSelectionVector( + expr, + hash_part.partition_count.try_into().unwrap(), + ))) + } + None => Ok(None), + } +} + + pub fn parse_protobuf_partitioning( partitioning: Option<&protobuf::Partitioning>, registry: &dyn FunctionRegistry, @@ -460,6 +485,14 @@ pub fn parse_protobuf_partitioning( input_schema, codec, ) + }, + Some(protobuf::partitioning::PartitionMethod::HashSelectionVector(hash_repartition)) => { + parse_protobuf_hash_selection_vector_partitioning( + Some(hash_repartition), + registry, + input_schema, + codec, + ) } Some(protobuf::partitioning::PartitionMethod::Unknown(partition_count)) => { Ok(Some(Partitioning::UnknownPartitioning( diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index d1b1f51ae107..98b18be90062 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -412,6 +412,17 @@ pub fn serialize_partitioning( }, )), } + }, + Partitioning::HashSelectionVector(exprs, partition_count) => { + let serialized_exprs = serialize_physical_exprs(exprs, codec)?; + protobuf::Partitioning { + partition_method: Some(protobuf::partitioning::PartitionMethod::HashSelectionVector( + protobuf::PhysicalHashSelectionVectorRepartition { + hash_expr: serialized_exprs, + partition_count: *partition_count as u64, + }, + )), + } } Partitioning::UnknownPartitioning(partition_count) => protobuf::Partitioning { partition_method: Some(protobuf::partitioning::PartitionMethod::Unknown( From 2e49946cff1d55a46ab79bfc274b38061ab44928 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 27 Mar 2025 01:29:18 +0800 Subject: [PATCH 05/37] add sqllogictests --- .../sqllogictest/test_files/aggregate.slt | 32 ++++++ .../test_files/information_schema.slt | 2 + .../sqllogictest/test_files/join.slt.part | 102 ++++++++++++++++++ 3 files changed, 136 insertions(+) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 552ec5e3b86a..cdddf2df74f4 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1119,6 +1119,38 @@ physical_plan 09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] 10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +statement ok +set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; + +query TT +explain select median(distinct c) from t; +---- +logical_plan +01)Projection: median(alias1) AS median(DISTINCT t.c) +02)--Aggregate: groupBy=[[]], aggr=[[median(alias1)]] +03)----Aggregate: groupBy=[[t.c AS alias1]], aggr=[[]] +04)------TableScan: t projection=[c] +physical_plan +01)ProjectionExec: expr=[median(alias1)@0 as median(DISTINCT t.c)] +02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=HashSelectionVector([alias1@0], 4), input_partitions=4 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] + +# TODO: it should be 3 after fix: https://github.com/apache/datafusion/issues/15383 +query I +select count(distinct c) from t; +---- +12 + +statement ok +set datafusion.optimizer.prefer_hash_selection_vector_partitioning = false; + statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 3a98a4d18523..6cbd6895a794 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -295,6 +295,7 @@ datafusion.optimizer.max_passes 3 datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_existing_union false datafusion.optimizer.prefer_hash_join true +datafusion.optimizer.prefer_hash_selection_vector_partitioning false datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 datafusion.optimizer.repartition_file_scans true @@ -405,6 +406,7 @@ datafusion.optimizer.max_passes 3 Number of times that the optimizer will attemp datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory +datafusion.optimizer.prefer_hash_selection_vector_partitioning false When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level datafusion.optimizer.repartition_file_min_size 10485760 Minimum total files size in bytes to perform file scan repartitioning. datafusion.optimizer.repartition_file_scans true When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 972dd2265343..58ad2665a001 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1389,6 +1389,108 @@ physical_plan 14)------------------FilterExec: y@1 = x@0 15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +query IIII rowsort +SELECT * FROM +(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) +JOIN f +ON col0 = f.a +JOIN s +ON col1 = s.b +---- +2 2 2 2 +3 3 3 3 + +statement ok +set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; + +query TT +explain +SELECT * FROM +(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) +JOIN f +ON col0 = f.a +JOIN s +ON col1 = s.b +---- +logical_plan +01)Inner Join: col1 = CAST(s.b AS Int64) +02)--Inner Join: col0 = CAST(f.a AS Int64) +03)----Projection: CAST(pairs.x AS Int64) + Int64(1) AS col0, CAST(pairs.y AS Int64) + Int64(1) AS col1 +04)------Filter: pairs.y = pairs.x +05)--------TableScan: pairs projection=[x, y] +06)----TableScan: f projection=[a] +07)--TableScan: s projection=[b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col1@1, CAST(s.b AS Int64)@1)], projection=[col0@0, col1@1, a@2, b@3] +03)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=HashSelectionVector([CAST(f.a AS Int64)@1], 16), input_partitions=1 +08)--------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------CoalesceBatchesExec: target_batch_size=8192 +11)------------RepartitionExec: partitioning=HashSelectionVector([col0@0], 16), input_partitions=16 +12)--------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +13)----------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------FilterExec: y@1 = x@0 +16)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +17)----CoalesceBatchesExec: target_batch_size=8192 +18)------RepartitionExec: partitioning=HashSelectionVector([CAST(s.b AS Int64)@1], 16), input_partitions=1 +19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] +20)----------DataSourceExec: partitions=1, partition_sizes=[1] + + +# TODO: It should be: +# 2 2 2 2 +# 3 3 3 3 +# After fix https://github.com/apache/datafusion/issues/15382 +query IIII +SELECT * FROM +(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) +JOIN f +ON col0 = f.a +JOIN s +ON col1 = s.b +---- +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 + +statement ok +set datafusion.optimizer.prefer_hash_selection_vector_partitioning = false; + statement ok drop table pairs; From 6cbe760e0c10a42ac0b991df34489d07c9c42651 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 27 Mar 2025 01:32:11 +0800 Subject: [PATCH 06/37] fix fmt and clippy --- datafusion/core/src/physical_planner.rs | 9 ++++++--- datafusion/physical-expr/src/partitioning.rs | 13 +++++++++---- .../src/enforce_distribution.rs | 8 ++++++-- datafusion/physical-plan/src/repartition/mod.rs | 7 ++----- .../proto/src/physical_plan/from_proto.rs | 17 ++++++++--------- datafusion/proto/src/physical_plan/to_proto.rs | 16 +++++++++------- 6 files changed, 40 insertions(+), 30 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index ffc960c15e34..2b9338ce5f60 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -804,10 +804,13 @@ impl DefaultPhysicalPlanner { ) }) .collect::>>()?; - if session_state.config_options().optimizer.prefer_hash_selection_vector_partitioning { + if session_state + .config_options() + .optimizer + .prefer_hash_selection_vector_partitioning + { Partitioning::HashSelectionVector(runtime_expr, *n) - } - else { + } else { Partitioning::Hash(runtime_expr, *n) } } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 70bcf23daa7d..f9ba9f7c730e 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -119,8 +119,8 @@ pub enum Partitioning { Hash(Vec>, usize), /// Allocate rows based on a hash of one of more expressions and the specified number /// of partitions with a selection vecotr column. - /// - /// The column is a boolean column called `selection` that is used to filter out rows + /// + /// The column is a boolean column called `selection` that is used to filter out rows /// that should not be included in the partition. `true` means the row should be included /// and `false` means the row should be excluded. HashSelectionVector(Vec>, usize), @@ -183,7 +183,8 @@ impl Partitioning { // Here we do not check the partition count for hash partitioning and assumes the partition count // and hash functions in the system are the same. In future if we plan to support storage partition-wise joins, // then we need to have the partition count and hash functions validation. - Partitioning::Hash(partition_exprs, _) | Partitioning::HashSelectionVector(partition_exprs, _ ) => { + Partitioning::Hash(partition_exprs, _) + | Partitioning::HashSelectionVector(partition_exprs, _) => { let fast_match = physical_exprs_equal(required_exprs, partition_exprs); // If the required exprs do not match, need to leverage the eq_properties provided by the child @@ -270,7 +271,11 @@ pub enum Distribution { impl Distribution { /// Creates a `Partitioning` that satisfies this `Distribution` - pub fn create_partitioning(self, partition_count: usize, prefer_selection_vector: bool) -> Partitioning { + pub fn create_partitioning( + self, + partition_count: usize, + prefer_selection_vector: bool, + ) -> Partitioning { match self { Distribution::UnspecifiedDistribution => { Partitioning::UnknownPartitioning(partition_count) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index c91331927cfa..be049b6fb768 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1272,8 +1272,12 @@ pub fn ensure_distribution( } // When inserting hash is necessary to satisfy hash requirement, insert hash repartition. if hash_necessary { - child = - add_hash_on_top(child, exprs.to_vec(), target_partitions, config.optimizer.prefer_hash_selection_vector_partitioning)?; + child = add_hash_on_top( + child, + exprs.to_vec(), + target_partitions, + config.optimizer.prefer_hash_selection_vector_partitioning, + )?; } } Distribution::UnspecifiedDistribution => { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index bc07c8bb1f5b..aee61dde296f 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -389,11 +389,8 @@ impl BatchPartitioner { .collect::>(); let new_col: ArrayRef = Arc::new(BooleanArray::from(select_vector)) as ArrayRef; - let mut columns = batch - .columns() - .iter() - .map(Arc::clone) - .collect::>(); + let mut columns = + batch.columns().iter().map(Arc::clone).collect::>(); columns.push(new_col); let mut options = RecordBatchOptions::new(); options = options.with_row_count(Some(batch.num_rows())); diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index b2ea5aef4dd1..5c7dfdbcd9b0 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -464,7 +464,6 @@ pub fn parse_protobuf_hash_selection_vector_partitioning( } } - pub fn parse_protobuf_partitioning( partitioning: Option<&protobuf::Partitioning>, registry: &dyn FunctionRegistry, @@ -485,15 +484,15 @@ pub fn parse_protobuf_partitioning( input_schema, codec, ) - }, - Some(protobuf::partitioning::PartitionMethod::HashSelectionVector(hash_repartition)) => { - parse_protobuf_hash_selection_vector_partitioning( - Some(hash_repartition), - registry, - input_schema, - codec, - ) } + Some(protobuf::partitioning::PartitionMethod::HashSelectionVector( + hash_repartition, + )) => parse_protobuf_hash_selection_vector_partitioning( + Some(hash_repartition), + registry, + input_schema, + codec, + ), Some(protobuf::partitioning::PartitionMethod::Unknown(partition_count)) => { Ok(Some(Partitioning::UnknownPartitioning( *partition_count as usize, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 98b18be90062..86df4466549e 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -412,16 +412,18 @@ pub fn serialize_partitioning( }, )), } - }, + } Partitioning::HashSelectionVector(exprs, partition_count) => { let serialized_exprs = serialize_physical_exprs(exprs, codec)?; protobuf::Partitioning { - partition_method: Some(protobuf::partitioning::PartitionMethod::HashSelectionVector( - protobuf::PhysicalHashSelectionVectorRepartition { - hash_expr: serialized_exprs, - partition_count: *partition_count as u64, - }, - )), + partition_method: Some( + protobuf::partitioning::PartitionMethod::HashSelectionVector( + protobuf::PhysicalHashSelectionVectorRepartition { + hash_expr: serialized_exprs, + partition_count: *partition_count as u64, + }, + ), + ), } } Partitioning::UnknownPartitioning(partition_count) => protobuf::Partitioning { From 25c3e2c5f92796468ee8eea1e11e3c77e52451f2 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 27 Mar 2025 01:44:47 +0800 Subject: [PATCH 07/37] rename column and address comment --- datafusion/physical-expr/src/partitioning.rs | 2 +- .../physical-plan/src/repartition/mod.rs | 20 +++++++++---------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index f9ba9f7c730e..837728bdef3e 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -120,7 +120,7 @@ pub enum Partitioning { /// Allocate rows based on a hash of one of more expressions and the specified number /// of partitions with a selection vecotr column. /// - /// The column is a boolean column called `selection` that is used to filter out rows + /// The column is a boolean column called `__selection` that is used to filter out rows /// that should not be included in the partition. `true` means the row should be included /// and `false` means the row should be excluded. HashSelectionVector(Vec>, usize), diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index aee61dde296f..544d76eb3209 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,9 +40,7 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{ - ArrayRef, BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions, -}; +use arrow::array::{BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow_schema::{DataType, Field}; @@ -208,7 +206,7 @@ enum BatchPartitionerState { }, } -pub static SELECTION_FILED_NAME: &str = "selection"; +pub static SELECTION_FILED_NAME: &str = "__selection"; impl BatchPartitioner { /// Create a new [`BatchPartitioner`] with the provided [`Partitioning`] @@ -383,15 +381,15 @@ impl BatchPartitioner { let it = (0..*num_partitions).map(move |partition| { // Tracking time required for repartitioned batches construction let _timer = partitioner_timer.timer(); - let select_vector = hash_vector - .iter() - .map(|&hash| hash == partition as u64) - .collect::>(); - let new_col: ArrayRef = - Arc::new(BooleanArray::from(select_vector)) as ArrayRef; + let selection_vector = Arc::new( + hash_vector + .iter() + .map(|&hash| Some(hash == partition as u64)) + .collect::(), + ); let mut columns = batch.columns().iter().map(Arc::clone).collect::>(); - columns.push(new_col); + columns.push(selection_vector); let mut options = RecordBatchOptions::new(); options = options.with_row_count(Some(batch.num_rows())); let batch = RecordBatch::try_new_with_options( From cb2606f945482e63064a82244809d4ab55a7497b Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Fri, 28 Mar 2025 00:16:59 +0800 Subject: [PATCH 08/37] fix config test --- docs/source/user-guide/configs.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index fe9c57857bef..9aa3c94f1833 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -114,6 +114,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | | datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.prefer_hash_selection_vector_partitioning | false | When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. | | datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | From 7e147e10f1ed6f4ef2ac1d6d95fe24ef5c76661f Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Fri, 28 Mar 2025 00:39:15 +0800 Subject: [PATCH 09/37] remove hash join test --- .../sqllogictest/test_files/join.slt.part | 102 ------------------ 1 file changed, 102 deletions(-) diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 58ad2665a001..972dd2265343 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1389,108 +1389,6 @@ physical_plan 14)------------------FilterExec: y@1 = x@0 15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] -query IIII rowsort -SELECT * FROM -(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) -JOIN f -ON col0 = f.a -JOIN s -ON col1 = s.b ----- -2 2 2 2 -3 3 3 3 - -statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; - -query TT -explain -SELECT * FROM -(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) -JOIN f -ON col0 = f.a -JOIN s -ON col1 = s.b ----- -logical_plan -01)Inner Join: col1 = CAST(s.b AS Int64) -02)--Inner Join: col0 = CAST(f.a AS Int64) -03)----Projection: CAST(pairs.x AS Int64) + Int64(1) AS col0, CAST(pairs.y AS Int64) + Int64(1) AS col1 -04)------Filter: pairs.y = pairs.x -05)--------TableScan: pairs projection=[x, y] -06)----TableScan: f projection=[a] -07)--TableScan: s projection=[b] -physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col1@1, CAST(s.b AS Int64)@1)], projection=[col0@0, col1@1, a@2, b@3] -03)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=HashSelectionVector([CAST(f.a AS Int64)@1], 16), input_partitions=1 -08)--------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] -09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)----------CoalesceBatchesExec: target_batch_size=8192 -11)------------RepartitionExec: partitioning=HashSelectionVector([col0@0], 16), input_partitions=16 -12)--------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] -13)----------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -14)------------------CoalesceBatchesExec: target_batch_size=8192 -15)--------------------FilterExec: y@1 = x@0 -16)----------------------DataSourceExec: partitions=1, partition_sizes=[1] -17)----CoalesceBatchesExec: target_batch_size=8192 -18)------RepartitionExec: partitioning=HashSelectionVector([CAST(s.b AS Int64)@1], 16), input_partitions=1 -19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] -20)----------DataSourceExec: partitions=1, partition_sizes=[1] - - -# TODO: It should be: -# 2 2 2 2 -# 3 3 3 3 -# After fix https://github.com/apache/datafusion/issues/15382 -query IIII -SELECT * FROM -(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) -JOIN f -ON col0 = f.a -JOIN s -ON col1 = s.b ----- -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 - -statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning = false; - statement ok drop table pairs; From 29fa0e69a5554bc45d78201321a789fc4a52083e Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Fri, 28 Mar 2025 09:48:30 +0800 Subject: [PATCH 10/37] fix typo --- datafusion/physical-expr/src/partitioning.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 837728bdef3e..280db202e107 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -114,11 +114,11 @@ use std::sync::Arc; pub enum Partitioning { /// Allocate batches using a round-robin algorithm and the specified number of partitions RoundRobinBatch(usize), - /// Allocate rows based on a hash of one of more expressions and the specified number of + /// Allocate rows based on a hash of one or more expressions and the specified number of /// partitions Hash(Vec>, usize), - /// Allocate rows based on a hash of one of more expressions and the specified number - /// of partitions with a selection vecotr column. + /// Allocate rows based on a hash of one or more expressions and the specified number + /// of partitions with a selection vector column. /// /// The column is a boolean column called `__selection` that is used to filter out rows /// that should not be included in the partition. `true` means the row should be included From 575af639c718cad60d6eeace665d96a57cf5d5c7 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Fri, 28 Mar 2025 21:44:55 +0800 Subject: [PATCH 11/37] add back the join test --- .../sqllogictest/test_files/join.slt.part | 138 ++++++++++++++++++ 1 file changed, 138 insertions(+) diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 972dd2265343..f512a7fb28d4 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1389,6 +1389,144 @@ physical_plan 14)------------------FilterExec: y@1 = x@0 15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] +# always use hash repartition +statement ok +set datafusion.optimizer.hash_join_single_partition_threshold = 0; + +query TT +explain +SELECT * FROM +(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) +JOIN f +ON col0 = f.a +JOIN s +ON col1 = s.b +---- +logical_plan +01)Inner Join: col1 = CAST(s.b AS Int64) +02)--Inner Join: col0 = CAST(f.a AS Int64) +03)----Projection: CAST(pairs.x AS Int64) + Int64(1) AS col0, CAST(pairs.y AS Int64) + Int64(1) AS col1 +04)------Filter: pairs.y = pairs.x +05)--------TableScan: pairs projection=[x, y] +06)----TableScan: f projection=[a] +07)--TableScan: s projection=[b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col1@1, CAST(s.b AS Int64)@1)], projection=[col0@0, col1@1, a@2, b@3] +03)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([CAST(f.a AS Int64)@1], 16), input_partitions=1 +08)--------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------CoalesceBatchesExec: target_batch_size=8192 +11)------------RepartitionExec: partitioning=Hash([col0@0], 16), input_partitions=16 +12)--------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +13)----------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------FilterExec: y@1 = x@0 +16)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +17)----CoalesceBatchesExec: target_batch_size=8192 +18)------RepartitionExec: partitioning=Hash([CAST(s.b AS Int64)@1], 16), input_partitions=1 +19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] +20)----------DataSourceExec: partitions=1, partition_sizes=[1] + +statement ok +set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; + +query TT +explain +SELECT * FROM +(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) +JOIN f +ON col0 = f.a +JOIN s +ON col1 = s.b +---- +logical_plan +01)Inner Join: col1 = CAST(s.b AS Int64) +02)--Inner Join: col0 = CAST(f.a AS Int64) +03)----Projection: CAST(pairs.x AS Int64) + Int64(1) AS col0, CAST(pairs.y AS Int64) + Int64(1) AS col1 +04)------Filter: pairs.y = pairs.x +05)--------TableScan: pairs projection=[x, y] +06)----TableScan: f projection=[a] +07)--TableScan: s projection=[b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col1@1, CAST(s.b AS Int64)@1)], projection=[col0@0, col1@1, a@2, b@3] +03)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=HashSelectionVector([CAST(f.a AS Int64)@1], 16), input_partitions=1 +08)--------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] +10)----------CoalesceBatchesExec: target_batch_size=8192 +11)------------RepartitionExec: partitioning=HashSelectionVector([col0@0], 16), input_partitions=16 +12)--------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +13)----------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +14)------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------FilterExec: y@1 = x@0 +16)----------------------DataSourceExec: partitions=1, partition_sizes=[1] +17)----CoalesceBatchesExec: target_batch_size=8192 +18)------RepartitionExec: partitioning=HashSelectionVector([CAST(s.b AS Int64)@1], 16), input_partitions=1 +19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] +20)----------DataSourceExec: partitions=1, partition_sizes=[1] + +# TODO: It should be: +# 2 2 2 2 +# 3 3 3 3 +# After fix https://github.com/apache/datafusion/issues/15382 +query IIII +SELECT * FROM +(SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) +JOIN f +ON col0 = f.a +JOIN s +ON col1 = s.b +---- +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 +2 2 2 2 +3 3 3 3 + +# reset the config +statement ok +set datafusion.optimizer.prefer_hash_selection_vector_partitioning = false; + +# reset the config +statement ok +set datafusion.optimizer.hash_join_single_partition_threshold = 1048576 + statement ok drop table pairs; From a789e5c75a0547719cb322c173a71329d5280a5a Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 00:49:11 +0800 Subject: [PATCH 12/37] add HashPartitionMode --- datafusion/core/src/physical_planner.rs | 4 +- .../physical_optimizer/projection_pushdown.rs | 23 ++++---- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/partitioning.rs | 56 +++++++++++++------ .../src/combine_partial_final_agg.rs | 2 +- .../src/enforce_distribution.rs | 24 ++++---- .../src/output_requirements.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 30 +++++++--- .../src/aggregates/no_grouping.rs | 4 +- .../physical-plan/src/aggregates/row_hash.rs | 4 +- .../physical-plan/src/joins/hash_join.rs | 12 +++- .../src/joins/sort_merge_join.rs | 6 +- .../src/joins/symmetric_hash_join.rs | 12 +++- .../src/windows/bounded_window_agg_exec.rs | 7 ++- .../src/windows/window_agg_exec.rs | 6 +- datafusion/proto/src/physical_plan/mod.rs | 10 +++- 16 files changed, 133 insertions(+), 73 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2b9338ce5f60..7e8cdaeb5a17 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -82,7 +82,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr::{HashPartitionMode, LexOrdering}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; @@ -742,7 +742,7 @@ impl DefaultPhysicalPlanner { let next_partition_mode = if can_repartition { // construct a second aggregation with 'AggregateMode::FinalPartitioned' - AggregateMode::FinalPartitioned + AggregateMode::FinalPartitioned(HashPartitionMode::HashPartitioned) } else { // construct a second aggregation, keeping the final column name equal to the // first aggregation and the expressions corresponding to the respective aggregate diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index e00a44188e57..6c56fc70c801 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -35,10 +35,10 @@ use datafusion_expr::{ use datafusion_physical_expr::expressions::{ binary, cast, col, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, }; -use datafusion_physical_expr::ScalarFunctionExpr; use datafusion_physical_expr::{ Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_expr::{HashPartitionMode, ScalarFunctionExpr}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::projection_pushdown::ProjectionPushdown; @@ -667,10 +667,10 @@ fn test_output_req_after_projection() -> Result<()> { options: Some(SortOptions::default()), }, ])), - Distribution::HashPartitioned(vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]), + Distribution::HashPartitioned( + vec![Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1))], + HashPartitionMode::HashPartitioned, + ), )); let projection: Arc = Arc::new(ProjectionExec::try_new( vec![ @@ -727,12 +727,13 @@ fn test_output_req_after_projection() -> Result<()> { Arc::new(Column::new("new_a", 1)), Arc::new(Column::new("b", 2)), ]; - if let Distribution::HashPartitioned(vec) = after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_distribution()[0] - .clone() + if let Distribution::HashPartitioned(vec, HashPartitionMode::HashPartitioned) = + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .required_input_distribution()[0] + .clone() { assert!(vec .iter() diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 9f795c81fa48..c6a533a0ac82 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -52,7 +52,7 @@ pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ calculate_union, AcrossPartitions, ConstExpr, EquivalenceProperties, }; -pub use partitioning::{Distribution, Partitioning}; +pub use partitioning::{Distribution, HashPartitionMode, Partitioning}; pub use physical_expr::{ create_ordering, create_physical_sort_expr, create_physical_sort_exprs, physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 280db202e107..932c963f6613 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -177,8 +177,8 @@ impl Partitioning { Distribution::UnspecifiedDistribution => true, Distribution::SinglePartition if self.partition_count() == 1 => true, // When partition count is 1, hash requirement is satisfied. - Distribution::HashPartitioned(_) if self.partition_count() == 1 => true, - Distribution::HashPartitioned(required_exprs) => { + Distribution::HashPartitioned(_, _) if self.partition_count() == 1 => true, + Distribution::HashPartitioned(required_exprs, _) => { match self { // Here we do not check the partition count for hash partitioning and assumes the partition count // and hash functions in the system are the same. In future if we plan to support storage partition-wise joins, @@ -266,27 +266,22 @@ pub enum Distribution { SinglePartition, /// Requires children to be distributed in such a way that the same /// values of the keys end up in the same partition - HashPartitioned(Vec>), + HashPartitioned(Vec>, HashPartitionMode), } impl Distribution { /// Creates a `Partitioning` that satisfies this `Distribution` - pub fn create_partitioning( - self, - partition_count: usize, - prefer_selection_vector: bool, - ) -> Partitioning { + pub fn create_partitioning(self, partition_count: usize) -> Partitioning { match self { Distribution::UnspecifiedDistribution => { Partitioning::UnknownPartitioning(partition_count) } Distribution::SinglePartition => Partitioning::UnknownPartitioning(1), - Distribution::HashPartitioned(expr) => { - if prefer_selection_vector { - Partitioning::HashSelectionVector(expr, partition_count) - } else { - Partitioning::Hash(expr, partition_count) - } + Distribution::HashPartitioned(expr, HashPartitionMode::HashPartitioned) => { + Partitioning::Hash(expr, partition_count) + } + Distribution::HashPartitioned(expr, HashPartitionMode::SelectionVector) => { + Partitioning::HashSelectionVector(expr, partition_count) } } } @@ -297,13 +292,35 @@ impl Display for Distribution { match self { Distribution::UnspecifiedDistribution => write!(f, "Unspecified"), Distribution::SinglePartition => write!(f, "SinglePartition"), - Distribution::HashPartitioned(exprs) => { - write!(f, "HashPartitioned[{}])", format_physical_expr_list(exprs)) + Distribution::HashPartitioned(exprs, mode) => { + write!( + f, + "HashPartitioned[{}, {}])", + format_physical_expr_list(exprs), + mode + ) } } } } +#[derive(Debug, Copy, Clone, PartialEq, Eq)] +pub enum HashPartitionMode { + /// Hash partitioning with a selection vector + SelectionVector, + /// The default hash partitioning + HashPartitioned, +} + +impl Display for HashPartitionMode { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + HashPartitionMode::SelectionVector => write!(f, "SelectionVector"), + HashPartitionMode::HashPartitioned => write!(f, "HashPartitioned"), + } + } +} + #[cfg(test)] mod tests { @@ -333,7 +350,10 @@ mod tests { let distribution_types = vec![ Distribution::UnspecifiedDistribution, Distribution::SinglePartition, - Distribution::HashPartitioned(partition_exprs1.clone()), + Distribution::HashPartitioned( + partition_exprs1.clone(), + HashPartitionMode::HashPartitioned, + ), ]; let single_partition = Partitioning::UnknownPartitioning(1); @@ -359,7 +379,7 @@ mod tests { Distribution::SinglePartition => { assert_eq!(result, (true, false, false, false, false)) } - Distribution::HashPartitioned(_) => { + Distribution::HashPartitioned(_, _) => { assert_eq!(result, (true, false, false, true, false)) } } diff --git a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs index 86f7e73e9e35..4c76737cd2f8 100644 --- a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs +++ b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs @@ -61,7 +61,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { if !matches!( agg_exec.mode(), - AggregateMode::Final | AggregateMode::FinalPartitioned + AggregateMode::Final | AggregateMode::FinalPartitioned(_) ) { return Ok(Transformed::no(plan)); } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index be049b6fb768..cfa1f06b4995 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -40,7 +40,8 @@ use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ - physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, + physical_exprs_equal, EquivalenceProperties, HashPartitionMode, PhysicalExpr, + PhysicalExprRef, }; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -392,7 +393,7 @@ pub fn adjust_input_keys_ordering( .map(Transformed::yes); } else if let Some(aggregate_exec) = plan.as_any().downcast_ref::() { if !requirements.data.is_empty() { - if aggregate_exec.mode() == &AggregateMode::FinalPartitioned { + if matches!(aggregate_exec.mode(), AggregateMode::FinalPartitioned(_)) { return reorder_aggregate_keys(requirements, aggregate_exec) .map(Transformed::yes); } else { @@ -521,7 +522,9 @@ pub fn reorder_aggregate_keys( .collect(), ); let new_final_agg = Arc::new(AggregateExec::try_new( - AggregateMode::FinalPartitioned, + AggregateMode::FinalPartitioned( + HashPartitionMode::HashPartitioned, + ), new_group_by, agg_exec.aggr_expr().to_vec(), agg_exec.filter_expr().to_vec(), @@ -889,7 +892,7 @@ fn add_hash_on_top( input: DistributionContext, hash_exprs: Vec>, n_target: usize, - prefer_selection_vector: bool, + mode: HashPartitionMode, ) -> Result { // Early return if hash repartition is unnecessary // `RepartitionExec: partitioning=Hash([...], 1), input_partitions=1` is unnecessary. @@ -897,7 +900,7 @@ fn add_hash_on_top( return Ok(input); } - let dist = Distribution::HashPartitioned(hash_exprs); + let dist = Distribution::HashPartitioned(hash_exprs, mode); let satisfied = input .plan .output_partitioning() @@ -914,7 +917,7 @@ fn add_hash_on_top( // requirements. // - Usage of order preserving variants is not desirable (per the flag // `config.optimizer.prefer_existing_sort`). - let partitioning = dist.create_partitioning(n_target, prefer_selection_vector); + let partitioning = dist.create_partitioning(n_target); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? .with_preserve_order(); @@ -1116,7 +1119,7 @@ fn get_repartition_requirement_status( Precision::Inexact(n_rows) => !should_use_estimates || (n_rows > batch_size), Precision::Absent => true, }; - let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); + let is_hash = matches!(requirement, Distribution::HashPartitioned(_, _)); // Hash re-partitioning is necessary when the input has more than one // partitions: let multi_partitions = child.output_partitioning().partition_count() > 1; @@ -1264,7 +1267,7 @@ pub fn ensure_distribution( Distribution::SinglePartition => { child = add_spm_on_top(child); } - Distribution::HashPartitioned(exprs) => { + Distribution::HashPartitioned(exprs, mode) => { if add_roundrobin { // Add round-robin repartitioning on top of the operator // to increase parallelism. @@ -1276,7 +1279,7 @@ pub fn ensure_distribution( child, exprs.to_vec(), target_partitions, - config.optimizer.prefer_hash_selection_vector_partitioning, + *mode, )?; } } @@ -1319,7 +1322,8 @@ pub fn ensure_distribution( // no ordering requirement match requirement { // Operator requires specific distribution. - Distribution::SinglePartition | Distribution::HashPartitioned(_) => { + Distribution::SinglePartition + | Distribution::HashPartitioned(_, _) => { // Since there is no ordering requirement, preserving ordering is pointless child = replace_order_preserving_variants(child)?; } diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 0488b3fd49a8..8932cec6aefb 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -232,7 +232,7 @@ impl ExecutionPlan for OutputRequirementExec { } let dist_req = match &self.required_input_distribution()[0] { - Distribution::HashPartitioned(exprs) => { + Distribution::HashPartitioned(exprs, mode) => { let mut updated_exprs = vec![]; for expr in exprs { let Some(new_expr) = update_expr(expr, projection.expr(), false)? @@ -241,7 +241,7 @@ impl ExecutionPlan for OutputRequirementExec { }; updated_exprs.push(new_expr); } - Distribution::HashPartitioned(updated_exprs) + Distribution::HashPartitioned(updated_exprs, *mode) } dist => dist.clone(), }; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a2616de5a62b..55af24c269d4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -41,6 +41,7 @@ use datafusion_common::{internal_err, not_impl_err, Constraint, Constraints, Res use datafusion_execution::TaskContext; use datafusion_expr::{Accumulator, Aggregate}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::HashPartitionMode; use datafusion_physical_expr::{ equivalence::ProjectionMapping, expressions::Column, physical_exprs_contains, ConstExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, @@ -94,7 +95,9 @@ pub enum AggregateMode { /// the same partitions, such as is the case with Hash repartitioning on the /// group keys. If a group key is duplicated, duplicate groups would be /// produced - FinalPartitioned, + /// + /// + FinalPartitioned(HashPartitionMode), /// *Single* layer of Aggregation, input is exactly one partition /// /// Applies the entire logical aggregation operation in a single operator, @@ -123,7 +126,7 @@ impl AggregateMode { AggregateMode::Partial | AggregateMode::Single | AggregateMode::SinglePartitioned => true, - AggregateMode::Final | AggregateMode::FinalPartitioned => false, + AggregateMode::Final | AggregateMode::FinalPartitioned(_) => false, } } } @@ -741,7 +744,7 @@ impl AggregateExec { // - aggregations sometimes also preserve invariants such as min, max... let column_statistics = Statistics::unknown_column(&self.schema()); match self.mode { - AggregateMode::Final | AggregateMode::FinalPartitioned + AggregateMode::Final | AggregateMode::FinalPartitioned(_) if self.group_by.expr.is_empty() => { Ok(Statistics { @@ -929,8 +932,17 @@ impl ExecutionPlan for AggregateExec { AggregateMode::Partial => { vec![Distribution::UnspecifiedDistribution] } - AggregateMode::FinalPartitioned | AggregateMode::SinglePartitioned => { - vec![Distribution::HashPartitioned(self.group_by.input_exprs())] + AggregateMode::SinglePartitioned => { + vec![Distribution::HashPartitioned( + self.group_by.input_exprs(), + HashPartitionMode::HashPartitioned, + )] + } + AggregateMode::FinalPartitioned(mode) => { + vec![Distribution::HashPartitioned( + self.group_by.input_exprs(), + *mode, + )] } AggregateMode::Final | AggregateMode::Single => { vec![Distribution::SinglePartition] @@ -1025,7 +1037,7 @@ fn create_schema( } } AggregateMode::Final - | AggregateMode::FinalPartitioned + | AggregateMode::FinalPartitioned(_) | AggregateMode::Single | AggregateMode::SinglePartitioned => { // in final mode, the field with the final result of the accumulator @@ -1232,7 +1244,7 @@ pub fn aggregate_expressions( }) .collect()), // In this mode, we build the merge expressions of the aggregation. - AggregateMode::Final | AggregateMode::FinalPartitioned => { + AggregateMode::Final | AggregateMode::FinalPartitioned(_) => { let mut col_idx_base = col_idx_base; aggr_expr .iter() @@ -1296,7 +1308,7 @@ pub fn finalize_aggregation( .collect() } AggregateMode::Final - | AggregateMode::FinalPartitioned + | AggregateMode::FinalPartitioned(_) | AggregateMode::Single | AggregateMode::SinglePartitioned => { // Merge the state to the final value @@ -2670,7 +2682,7 @@ mod tests { None, )?; let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::FinalPartitioned, + AggregateMode::FinalPartitioned(HashPartitionMode::HashPartitioned), group_by, aggr_expr, vec![None], diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 9474a5f88c92..5d68778f9858 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -82,7 +82,7 @@ impl AggregateStream { AggregateMode::Partial | AggregateMode::Single | AggregateMode::SinglePartitioned => agg_filter_expr, - AggregateMode::Final | AggregateMode::FinalPartitioned => { + AggregateMode::Final | AggregateMode::FinalPartitioned(_) => { vec![None; agg.aggr_expr.len()] } }; @@ -233,7 +233,7 @@ fn aggregate_batch( AggregateMode::Partial | AggregateMode::Single | AggregateMode::SinglePartitioned => accum.update_batch(&values), - AggregateMode::Final | AggregateMode::FinalPartitioned => { + AggregateMode::Final | AggregateMode::FinalPartitioned(_) => { accum.merge_batch(&values) } }; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 232565a04466..60bf38897e68 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -472,7 +472,7 @@ impl GroupedHashAggregateStream { AggregateMode::Partial | AggregateMode::Single | AggregateMode::SinglePartitioned => agg_filter_expr, - AggregateMode::Final | AggregateMode::FinalPartitioned => { + AggregateMode::Final | AggregateMode::FinalPartitioned(_) => { vec![None; agg.aggr_expr.len()] } }; @@ -956,7 +956,7 @@ impl GroupedHashAggregateStream { output.extend(acc.state(emit_to)?) } AggregateMode::Final - | AggregateMode::FinalPartitioned + | AggregateMode::FinalPartitioned(_) | AggregateMode::Single | AggregateMode::SinglePartitioned => output.push(acc.evaluate(emit_to)?), } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 3a2c19310176..c34128df4010 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -78,7 +78,7 @@ use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::{ join_equivalence_properties, ProjectionMapping, }; -use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr::{HashPartitionMode, PhysicalExprRef}; use datafusion_physical_expr_common::datum::compare_op_for_nested; use ahash::RandomState; @@ -718,8 +718,14 @@ impl ExecutionPlan for HashJoinExec { .map(|(l, r)| (Arc::clone(l), Arc::clone(r))) .unzip(); vec![ - Distribution::HashPartitioned(left_expr), - Distribution::HashPartitioned(right_expr), + Distribution::HashPartitioned( + left_expr, + HashPartitionMode::HashPartitioned, + ), + Distribution::HashPartitioned( + right_expr, + HashPartitionMode::HashPartitioned, + ), ] } PartitionMode::Auto => vec![ diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 4ad0d434c188..50c49e17deaf 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -71,7 +71,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr::{HashPartitionMode, PhysicalExprRef}; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; @@ -405,8 +405,8 @@ impl ExecutionPlan for SortMergeJoinExec { .map(|(l, r)| (Arc::clone(l), Arc::clone(r))) .unzip(); vec![ - Distribution::HashPartitioned(left_expr), - Distribution::HashPartitioned(right_expr), + Distribution::HashPartitioned(left_expr, HashPartitionMode::HashPartitioned), + Distribution::HashPartitioned(right_expr, HashPartitionMode::HashPartitioned), ] } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 0dcb42169e00..d599edd12240 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -73,7 +73,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; -use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr::{HashPartitionMode, PhysicalExprRef}; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; @@ -423,8 +423,14 @@ impl ExecutionPlan for SymmetricHashJoinExec { .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _)) .unzip(); vec![ - Distribution::HashPartitioned(left_expr), - Distribution::HashPartitioned(right_expr), + Distribution::HashPartitioned( + left_expr, + HashPartitionMode::HashPartitioned, + ), + Distribution::HashPartitioned( + right_expr, + HashPartitionMode::HashPartitioned, + ), ] } StreamJoinPartitionMode::SinglePartition => { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 1ea02adafe3f..820bda865f68 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -60,7 +60,7 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{HashPartitionMode, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::stream::Stream; @@ -318,7 +318,10 @@ impl ExecutionPlan for BoundedWindowAggExec { debug!("No partition defined for BoundedWindowAggExec!!!"); vec![Distribution::SinglePartition] } else { - vec![Distribution::HashPartitioned(self.partition_keys().clone())] + vec![Distribution::HashPartitioned( + self.partition_keys().clone(), + HashPartitionMode::HashPartitioned, + )] } } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 4c76e2230875..dce390ed0e5a 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -44,6 +44,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::HashPartitionMode; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; @@ -252,7 +253,10 @@ impl ExecutionPlan for WindowAggExec { if self.partition_keys().is_empty() { vec![Distribution::SinglePartition] } else { - vec![Distribution::HashPartitioned(self.partition_keys())] + vec![Distribution::HashPartitioned( + self.partition_keys(), + HashPartitionMode::HashPartitioned, + )] } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 9b636d15103d..8cf37bf30115 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -58,7 +58,9 @@ use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; -use datafusion::physical_expr::{LexOrdering, LexRequirement, PhysicalExprRef}; +use datafusion::physical_expr::{ + HashPartitionMode, LexOrdering, LexRequirement, PhysicalExprRef, +}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -933,7 +935,9 @@ impl protobuf::PhysicalPlanNode { let agg_mode: AggregateMode = match mode { protobuf::AggregateMode::Partial => AggregateMode::Partial, protobuf::AggregateMode::Final => AggregateMode::Final, - protobuf::AggregateMode::FinalPartitioned => AggregateMode::FinalPartitioned, + protobuf::AggregateMode::FinalPartitioned => AggregateMode::FinalPartitioned( + HashPartitionMode::HashPartitioned, + ), protobuf::AggregateMode::Single => AggregateMode::Single, protobuf::AggregateMode::SinglePartitioned => { AggregateMode::SinglePartitioned @@ -2148,7 +2152,7 @@ impl protobuf::PhysicalPlanNode { let agg_mode = match exec.mode() { AggregateMode::Partial => protobuf::AggregateMode::Partial, AggregateMode::Final => protobuf::AggregateMode::Final, - AggregateMode::FinalPartitioned => protobuf::AggregateMode::FinalPartitioned, + AggregateMode::FinalPartitioned(_) => protobuf::AggregateMode::FinalPartitioned, AggregateMode::Single => protobuf::AggregateMode::Single, AggregateMode::SinglePartitioned => { protobuf::AggregateMode::SinglePartitioned From 13d9bb88e41e71bf70631ba2fa3587ebcbca3327 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 01:08:35 +0800 Subject: [PATCH 13/37] update sqllogictests --- datafusion/core/src/physical_planner.rs | 7 +- .../test_files/agg_func_substitute.slt | 6 +- .../sqllogictest/test_files/aggregate.slt | 14 ++-- .../test_files/aggregates_topk.slt | 14 ++-- .../test_files/count_star_rule.slt | 4 +- .../sqllogictest/test_files/distinct_on.slt | 2 +- .../sqllogictest/test_files/explain_tree.slt | 82 +++++++++++++++++++ .../sqllogictest/test_files/group_by.slt | 38 ++++----- .../sqllogictest/test_files/join.slt.part | 36 +------- datafusion/sqllogictest/test_files/joins.slt | 8 +- datafusion/sqllogictest/test_files/limit.slt | 4 +- datafusion/sqllogictest/test_files/order.slt | 4 +- .../sqllogictest/test_files/repartition.slt | 4 +- datafusion/sqllogictest/test_files/select.slt | 2 +- .../sqllogictest/test_files/subquery.slt | 8 +- .../sqllogictest/test_files/subquery_sort.slt | 2 +- .../test_files/tpch/plans/q1.slt.part | 2 +- .../test_files/tpch/plans/q10.slt.part | 2 +- .../test_files/tpch/plans/q11.slt.part | 2 +- .../test_files/tpch/plans/q12.slt.part | 2 +- .../test_files/tpch/plans/q13.slt.part | 2 +- .../test_files/tpch/plans/q15.slt.part | 4 +- .../test_files/tpch/plans/q16.slt.part | 4 +- .../test_files/tpch/plans/q17.slt.part | 2 +- .../test_files/tpch/plans/q18.slt.part | 4 +- .../test_files/tpch/plans/q2.slt.part | 2 +- .../test_files/tpch/plans/q20.slt.part | 2 +- .../test_files/tpch/plans/q21.slt.part | 2 +- .../test_files/tpch/plans/q22.slt.part | 2 +- .../test_files/tpch/plans/q3.slt.part | 2 +- .../test_files/tpch/plans/q4.slt.part | 2 +- .../test_files/tpch/plans/q5.slt.part | 2 +- .../test_files/tpch/plans/q7.slt.part | 2 +- .../test_files/tpch/plans/q8.slt.part | 2 +- .../test_files/tpch/plans/q9.slt.part | 2 +- datafusion/sqllogictest/test_files/union.slt | 16 ++-- datafusion/sqllogictest/test_files/window.slt | 4 +- 37 files changed, 178 insertions(+), 121 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 7e8cdaeb5a17..4902756ca414 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -741,8 +741,13 @@ impl DefaultPhysicalPlanner { let updated_aggregates = initial_aggr.aggr_expr().to_vec(); let next_partition_mode = if can_repartition { + let mode = if session_state.config_options().optimizer.prefer_hash_selection_vector_partitioning { + HashPartitionMode::SelectionVector + } else { + HashPartitionMode::HashPartitioned + }; // construct a second aggregation with 'AggregateMode::FinalPartitioned' - AggregateMode::FinalPartitioned(HashPartitionMode::HashPartitioned) + AggregateMode::FinalPartitioned(mode) } else { // construct a second aggregation, keeping the final column name equal to the // first aggregation and the expressions corresponding to the respective aggregate diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 9aeaaacb1071..080014a0d15d 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -44,7 +44,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 @@ -64,7 +64,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 @@ -83,7 +83,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index cdddf2df74f4..909ce117e23c 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -374,7 +374,7 @@ logical_plan 15)------------EmptyRelation physical_plan 01)ProjectionExec: expr=[array_length(array_agg(DISTINCT a.foo)@1) as array_length(array_agg(DISTINCT a.foo)), sum(DISTINCT Int64(1))@2 as sum(DISTINCT Int64(1))] -02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([id@0], 4), input_partitions=5 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted @@ -1112,7 +1112,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -1135,7 +1135,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(SelectionVector), gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=HashSelectionVector([alias1@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -5282,11 +5282,11 @@ logical_plan 04)------TableScan: aggregate_test_100 projection=[c1, c3] physical_plan 01)CoalescePartitionsExec: fetch=5 -02)--AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 05)--------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -06)----------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] +06)----------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] @@ -6387,7 +6387,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] -03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -6536,7 +6536,7 @@ logical_plan 15)------------EmptyRelation physical_plan 01)ProjectionExec: expr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST]@1 as last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))@2 as sum(DISTINCT Int64(1))] -02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([id@0], 4), input_partitions=5 05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[last_value(a.foo) ORDER BY [a.foo ASC NULLS LAST], sum(DISTINCT Int64(1))], ordering_mode=Sorted diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index cc1693843848..3d708f011ee7 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -45,7 +45,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -109,7 +109,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -126,7 +126,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [min(traces.timestamp)@1 DESC], fetch=4 02)--SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -143,7 +143,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [max(traces.timestamp)@1 ASC NULLS LAST], fetch=4 02)--SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -160,7 +160,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 02)--SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -234,7 +234,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [max(traces_utf8view.timestamp)@1 DESC], fetch=4 02)--SortExec: TopK(fetch=4), expr=[max(traces_utf8view.timestamp)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[max(traces_utf8view.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -262,7 +262,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [max(traces_largeutf8.timestamp)@1 DESC], fetch=4 02)--SortExec: TopK(fetch=4), expr=[max(traces_largeutf8.timestamp)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[trace_id@0 as trace_id], aggr=[max(traces_largeutf8.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index d38d3490fed4..61fafe8b04d1 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -48,7 +48,7 @@ logical_plan 03)----TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as count()] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[count(Int64(1))] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -67,7 +67,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as cnt] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: count(Int64(1))@1 > 0 -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[count(Int64(1))] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index b4a491619e89..f0c0ffe95490 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -97,7 +97,7 @@ physical_plan 01)ProjectionExec: expr=[first_value(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, first_value(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] 02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 03)----SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[first_value(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], first_value(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1], aggr=[first_value(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], first_value(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], first_value(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 15bf61576571..ee450a821632 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -248,6 +248,48 @@ physical_plan 48)│ files: 1 │ 49)│ format: csv │ 50)└───────────────────────────┘ +11)│ (HashPartitioned │ +12)│ ) │ +13)└─────────────┬─────────────┘ +14)┌─────────────┴─────────────┐ +15)│ CoalesceBatchesExec │ +16)│ -------------------- │ +17)│ target_batch_size: │ +18)│ 8192 │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ RepartitionExec │ +22)│ -------------------- │ +23)│ input_partition_count: │ +24)│ 4 │ +25)│ │ +26)│ partitioning_scheme: │ +27)│ Hash([string_col@0], 4) │ +28)└─────────────┬─────────────┘ +29)┌─────────────┴─────────────┐ +30)│ AggregateExec │ +31)│ -------------------- │ +32)│ aggr: │ +33)│ sum(table1.bigint_col) │ +34)│ │ +35)│ group_by: string_col │ +36)│ mode: Partial │ +37)└─────────────┬─────────────┘ +38)┌─────────────┴─────────────┐ +39)│ RepartitionExec │ +40)│ -------------------- │ +41)│ input_partition_count: │ +42)│ 1 │ +43)│ │ +44)│ partitioning_scheme: │ +45)│ RoundRobinBatch(4) │ +46)└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐ +48)│ DataSourceExec │ +49)│ -------------------- │ +50)│ files: 1 │ +51)│ format: csv │ +52)└───────────────────────────┘ # Limit @@ -1533,6 +1575,46 @@ physical_plan 61)│ format: memory ││ format: memory │ 62)│ rows: 1 ││ rows: 1 │ 63)└───────────────────────────┘└───────────────────────────┘ +26)│ (HashPartitioned ││ (HashPartitioned │ +27)│ ) ││ ) │ +28)└─────────────┬─────────────┘└─────────────┬─────────────┘ +29)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +30)│ CoalesceBatchesExec ││ CoalesceBatchesExec │ +31)│ -------------------- ││ -------------------- │ +32)│ target_batch_size: ││ target_batch_size: │ +33)│ 8192 ││ 8192 │ +34)└─────────────┬─────────────┘└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +36)│ RepartitionExec ││ RepartitionExec │ +37)│ -------------------- ││ -------------------- │ +38)│ input_partition_count: ││ input_partition_count: │ +39)│ 4 ││ 4 │ +40)│ ││ │ +41)│ partitioning_scheme: ││ partitioning_scheme: │ +42)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ +43)└─────────────┬─────────────┘└─────────────┬─────────────┘ +44)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +45)│ RepartitionExec ││ RepartitionExec │ +46)│ -------------------- ││ -------------------- │ +47)│ input_partition_count: ││ input_partition_count: │ +48)│ 1 ││ 1 │ +49)│ ││ │ +50)│ partitioning_scheme: ││ partitioning_scheme: │ +51)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ +52)└─────────────┬─────────────┘└─────────────┬─────────────┘ +53)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +54)│ AggregateExec ││ AggregateExec │ +55)│ -------------------- ││ -------------------- │ +56)│ group_by: name ││ group_by: name │ +57)│ mode: Partial ││ mode: Partial │ +58)└─────────────┬─────────────┘└─────────────┬─────────────┘ +59)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +60)│ DataSourceExec ││ DataSourceExec │ +61)│ -------------------- ││ -------------------- │ +62)│ bytes: 1320 ││ bytes: 1312 │ +63)│ format: memory ││ format: memory │ +64)│ rows: 1 ││ rows: 1 │ +65)└───────────────────────────┘└───────────────────────────┘ # Test explain tree for UnionExec query TT diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 9e67018ecd0b..fa4bac74b3ee 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2016,7 +2016,7 @@ physical_plan 01)SortPreservingMergeExec: [col0@0 ASC NULLS LAST] 02)--SortExec: expr=[col0@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[col0@0 as col0, last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] @@ -2983,7 +2983,7 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -3019,7 +3019,7 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -3183,7 +3183,7 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] @@ -3219,7 +3219,7 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] @@ -3420,7 +3420,7 @@ physical_plan 01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] 02)--SortExec: expr=[sn@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[sn@0 as sn, amount@1 as amount, 2 * CAST(sn@0 AS Int64) as Int64(2) * s.sn] -04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[sn@0 as sn, amount@1 as amount], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] @@ -3489,7 +3489,7 @@ physical_plan 01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] 02)--SortExec: expr=[sn@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[sn@0 as sn, sum(l.amount)@2 as sum(l.amount), amount@1 as amount] -04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[sum(l.amount)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[sn@0 as sn, amount@1 as amount], aggr=[sum(l.amount)] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)] @@ -3636,7 +3636,7 @@ physical_plan 01)SortPreservingMergeExec: [sn@2 ASC NULLS LAST] 02)--SortExec: expr=[sn@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] -04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] @@ -3864,7 +3864,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan 01)ProjectionExec: expr=[first_value(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, last_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[first_value(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], last_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[d@0 as d], aggr=[first_value(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], last_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[first_value(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], first_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] @@ -3975,7 +3975,7 @@ logical_plan 01)Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[sum(CAST(multiple_ordered_table_with_pk.d AS Int64))]] 02)--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c@0 as c, b@1 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 02)--SortExec: expr=[c@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 @@ -4016,7 +4016,7 @@ logical_plan 01)Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[sum(CAST(multiple_ordered_table_with_pk.d AS Int64))]] 02)--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c@0 as c, b@1 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 02)--SortExec: expr=[c@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 @@ -4239,7 +4239,7 @@ logical_plan 03)----TableScan: t1 projection=[x, y] physical_plan 01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), max(DISTINCT t1.x)@2 as max(DISTINCT t1.x)] -02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -4256,11 +4256,11 @@ logical_plan 04)------TableScan: t1 projection=[x, y] physical_plan 01)ProjectionExec: expr=[sum(alias1)@1 as sum(DISTINCT t1.x), max(alias1)@2 as max(DISTINCT t1.x)] -02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] -06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +06)----------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[y@0 as y, alias1@1 as alias1], aggr=[] 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -4293,7 +4293,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 02)--ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] -03)----AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC 06)----------AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted @@ -4348,7 +4348,7 @@ physical_plan 01)SortPreservingMergeExec: [months@0 DESC], fetch=5 02)--SortExec: TopK(fetch=5), expr=[months@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as months] -04)------AggregateExec: mode=FinalPartitioned, gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] @@ -4460,11 +4460,11 @@ physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c1@0 as c1, count(alias1)@1 as count(DISTINCT aggregate_test_100.c2), min(alias1)@2 as min(DISTINCT aggregate_test_100.c2), sum(alias2)@3 as sum(aggregate_test_100.c3), max(alias3)@4 as max(aggregate_test_100.c4)] -04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] +08)--------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] @@ -4637,7 +4637,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [max(timestamp_table.t1)@1 DESC], fetch=4 02)--SortExec: TopK(fetch=4), expr=[max(timestamp_table.t1)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[max(timestamp_table.t1)], lim=[4] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c2@0 as c2], aggr=[max(timestamp_table.t1)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 06)----------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[max(timestamp_table.t1)], lim=[4] diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index f512a7fb28d4..66ced5ff8ffa 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1459,18 +1459,18 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=HashSelectionVector([CAST(f.a AS Int64)@1], 16), input_partitions=1 +07)------------RepartitionExec: partitioning=Hash([CAST(f.a AS Int64)@1], 16), input_partitions=1 08)--------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)----------CoalesceBatchesExec: target_batch_size=8192 -11)------------RepartitionExec: partitioning=HashSelectionVector([col0@0], 16), input_partitions=16 +11)------------RepartitionExec: partitioning=Hash([col0@0], 16), input_partitions=16 12)--------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] 13)----------------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 14)------------------CoalesceBatchesExec: target_batch_size=8192 15)--------------------FilterExec: y@1 = x@0 16)----------------------DataSourceExec: partitions=1, partition_sizes=[1] 17)----CoalesceBatchesExec: target_batch_size=8192 -18)------RepartitionExec: partitioning=HashSelectionVector([CAST(s.b AS Int64)@1], 16), input_partitions=1 +18)------RepartitionExec: partitioning=Hash([CAST(s.b AS Int64)@1], 16), input_partitions=1 19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] 20)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1486,38 +1486,8 @@ ON col0 = f.a JOIN s ON col1 = s.b ---- -2 2 2 2 3 3 3 3 2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 -2 2 2 2 -3 3 3 3 # reset the config statement ok diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ddf701ba04ef..c154ffc5b15e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1338,7 +1338,7 @@ logical_plan 04)------TableScan: join_t1 projection=[t1_id] 05)------TableScan: join_t2 projection=[t2_id] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[] +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t1_id@0 as t1_id], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=2 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] @@ -1396,7 +1396,7 @@ logical_plan 06)--------TableScan: join_t2 projection=[t2_id] physical_plan 01)ProjectionExec: expr=[count(Int64(1))@1 as count(*)] -02)--AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] @@ -1425,7 +1425,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[count(alias1)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 08)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] @@ -3448,7 +3448,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC] 02)--SortExec: expr=[a@0 ASC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[a@0 as a, last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]] diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 93ffa313b8f7..6e03911f84fa 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -404,7 +404,7 @@ logical_plan 01)Aggregate: groupBy=[[t1000.i]], aggr=[[]] 02)--TableScan: t1000 projection=[i] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[i@0 as i], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -636,7 +636,7 @@ physical_plan 01)GlobalLimitExec: skip=1, fetch=2 02)--SortPreservingMergeExec: [b@0 DESC], fetch=3 03)----SortExec: TopK(fetch=3), expr=[b@0 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[sum(ordered_table.a)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b], aggr=[sum(ordered_table.a)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 7d9bea3d2b6f..a68e7c91ac82 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -793,7 +793,7 @@ physical_plan 02)--SortExec: expr=[m@0 ASC NULLS LAST, t@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----InterleaveExec 04)------ProjectionExec: expr=[0 as m, t@0 as t] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t@0 as t], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -801,7 +801,7 @@ physical_plan 10)------------------ProjectionExec: expr=[column1@0 as t] 11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] 12)------ProjectionExec: expr=[1 as m, t@0 as t] -13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +13)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t@0 as t], aggr=[] 14)----------CoalesceBatchesExec: target_batch_size=8192 15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 70666346e2ca..9dc917c64452 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -43,7 +43,7 @@ logical_plan 01)Aggregate: groupBy=[[parquet_table.column1]], aggr=[[sum(CAST(parquet_table.column2 AS Int64))]] 02)--TableScan: parquet_table projection=[column1, column2] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] @@ -61,7 +61,7 @@ logical_plan 01)Aggregate: groupBy=[[parquet_table.column1]], aggr=[[sum(CAST(parquet_table.column2 AS Int64))]] 02)--TableScan: parquet_table projection=[column1, column2] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index aa14faf984e4..ffe8813eb4f4 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1549,7 +1549,7 @@ logical_plan 06)----------TableScan: aggregate_test_100 projection=[c1, c2] physical_plan 01)ProjectionExec: expr=[c2@0 as c2, count(Int64(1))@1 as count(*)] -02)--AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[count(Int64(1))] +02)--AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c2@0 as c2], aggr=[count(Int64(1))] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 05)--------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index a0ac15b740d7..281e4d54d1e9 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -205,7 +205,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +06)----------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] @@ -240,7 +240,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int * Float64(1))@1 + 1 as sum(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] +06)----------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] @@ -275,7 +275,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +06)----------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] @@ -313,7 +313,7 @@ physical_plan 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------FilterExec: sum(t2.t2_int)@1 < 3 -08)--------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +08)--------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index 5d22bf92e7e6..49c371fef51b 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -150,7 +150,7 @@ physical_plan 02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c3@2 DESC, c9@3 ASC NULLS LAST] 03)----SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 DESC, c9@3 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@1 as c1, first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@2 as c2, first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@3 as c3, first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@4 as c9] -05)--------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 4a6ad5eddfb7..decf36b54cd4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -50,7 +50,7 @@ physical_plan 01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] 02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(Int64(1))@9 as count_order] -04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index fee496f92055..71ae79421bb2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -72,7 +72,7 @@ physical_plan 01)SortPreservingMergeExec: [revenue@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[revenue@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] -04)------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 1dba8c053720..cb7c13a5e632 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -77,7 +77,7 @@ physical_plan 02)--ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] 03)----NestedLoopJoinExec: join_type=Inner, filter=CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1, projection=[ps_partkey@0, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1] 04)------CoalescePartitionsExec -05)--------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index 3757fc48dba0..1bae4493e160 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -62,7 +62,7 @@ physical_plan 01)SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] 02)--SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] -04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index e9d9cf141d10..580ceacd3fb8 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -56,7 +56,7 @@ physical_plan 01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(Int64(1))@1 as custdist] -04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c_count@0 as c_count], aggr=[count(Int64(1))] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 0636a033b25a..60f6d4c54075 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -79,7 +79,7 @@ physical_plan 06)----------CoalescePartitionsExec 07)------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] 08)--------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)----------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 12)----------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] @@ -93,7 +93,7 @@ physical_plan 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_name, s_address, s_phone], file_type=csv, has_header=false 22)------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -23)--------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +23)--------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 24)----------------CoalesceBatchesExec: target_batch_size=8192 25)------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 26)--------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index c648f164c809..39ddee932b46 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -68,11 +68,11 @@ physical_plan 01)SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] -04)------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] +08)--------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 02553890bcf5..118b5eb9704c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -69,7 +69,7 @@ physical_plan 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_container], file_type=csv, has_header=false 18)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] -19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +19)--------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] 20)----------------CoalesceBatchesExec: target_batch_size=8192 21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 55da5371671e..2703e2418cab 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -69,7 +69,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] 02)--SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] +03)----AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] @@ -93,7 +93,7 @@ physical_plan 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_quantity], file_type=csv, has_header=false 25)----------------CoalesceBatchesExec: target_batch_size=8192 26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +27)--------------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] 28)----------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index 2a8ee9f229b7..f6563ca31fc0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -147,7 +147,7 @@ physical_plan 46)----------CoalesceBatchesExec: target_batch_size=8192 47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +49)----------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 50)------------------CoalesceBatchesExec: target_batch_size=8192 51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 4844d5fae60b..70b0cbd1192f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -117,7 +117,7 @@ physical_plan 34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 35)----------------------------------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 36)----------------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] -37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +37)------------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] 38)--------------------CoalesceBatchesExec: target_batch_size=8192 39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index bb3e884e27be..96c4554d2bfe 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -93,7 +93,7 @@ physical_plan 01)SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[s_name@0 as s_name, count(Int64(1))@1 as numwait] -04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[s_name@0 as s_name], aggr=[count(Int64(1))] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 828bf967d8f4..860f06a8f7d3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -77,7 +77,7 @@ physical_plan 01)SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] 02)--SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] -04)------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 2ad496ef26fd..0fe2438686eb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -61,7 +61,7 @@ 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=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)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), 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)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index f7de3cd3c967..48125f3fcc32 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -56,7 +56,7 @@ physical_plan 01)SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] 02)--SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(Int64(1))@1 as order_count] -04)------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index f192f987b3ef..ee0a8825371f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -70,7 +70,7 @@ physical_plan 01)SortPreservingMergeExec: [revenue@1 DESC] 02)--SortExec: expr=[revenue@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[n_name@0 as n_name, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] -04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index e03de9596fbe..546531524f95 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -87,7 +87,7 @@ physical_plan 01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] 02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] -04)------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 88ceffd62ad3..0d2f9c4ab7b2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -93,7 +93,7 @@ physical_plan 01)SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] 02)--SortExec: expr=[o_year@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[o_year@0 as o_year, CAST(CAST(sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 AS Decimal128(12, 2)) / CAST(sum(all_nations.volume)@2 AS Decimal128(12, 2)) AS Decimal128(15, 2)) as mkt_share] -04)------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 8ccf967187d7..2232b36f8643 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -78,7 +78,7 @@ physical_plan 01)SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] -04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 356f1598bc0f..10f12f240ff9 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -233,7 +233,7 @@ logical_plan 05)----Projection: t2.name || Utf8("_new") AS name 06)------TableScan: t2 projection=[name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +01)AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 @@ -310,7 +310,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] 04)------CoalescePartitionsExec -05)--------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[id@0 as id, name@1 as name], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] @@ -324,7 +324,7 @@ physical_plan 16)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] 17)--------CoalescePartitionsExec 18)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -19)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +19)------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[id@0 as id, name@1 as name], aggr=[] 20)--------------CoalesceBatchesExec: target_batch_size=2 21)----------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 22)------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] @@ -380,7 +380,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] 04)------CoalescePartitionsExec -05)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[name@0 as name], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] @@ -391,7 +391,7 @@ physical_plan 13)--CoalesceBatchesExec: target_batch_size=2 14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] 15)------CoalescePartitionsExec -16)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +16)--------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[name@0 as name], aggr=[] 17)----------CoalesceBatchesExec: target_batch_size=2 18)------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 19)--------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] @@ -453,13 +453,13 @@ physical_plan 01)ProjectionExec: expr=[count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=SinglePartitioned, gby=[name@0 as name], aggr=[count(Int64(1))] 03)----InterleaveExec -04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[name@0 as name], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] -10)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +10)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[name@0 as name], aggr=[] 11)--------CoalesceBatchesExec: target_batch_size=2 12)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 13)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] @@ -511,7 +511,7 @@ physical_plan 05)--------CoalescePartitionsExec 06)----------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 07)------------ProjectionExec: expr=[] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +08)--------------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1], aggr=[] 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 52cc80eae1c8..4633276229cb 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -274,7 +274,7 @@ physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[b@0 as b, max(d.a)@1 as max_a] -04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[max(d.a)] +04)------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b], aggr=[max(d.a)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted @@ -1778,7 +1778,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 05)--------ProjectionExec: expr=[] -06)----------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +06)----------AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1], aggr=[] 07)------------CoalesceBatchesExec: target_batch_size=4096 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] From 316998d568f9be50bd8c72a931d4f5c93396d593 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 01:48:17 +0800 Subject: [PATCH 14/37] fix fmt --- datafusion/core/src/physical_planner.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 4902756ca414..a2f4c3ce8b43 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -741,7 +741,11 @@ impl DefaultPhysicalPlanner { let updated_aggregates = initial_aggr.aggr_expr().to_vec(); let next_partition_mode = if can_repartition { - let mode = if session_state.config_options().optimizer.prefer_hash_selection_vector_partitioning { + let mode = if session_state + .config_options() + .optimizer + .prefer_hash_selection_vector_partitioning + { HashPartitionMode::SelectionVector } else { HashPartitionMode::HashPartitioned From adc00d87829ba25f1a7f93d3b03fa2ea82804caa Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 02:01:46 +0800 Subject: [PATCH 15/37] add todo comment --- datafusion/sqllogictest/test_files/join.slt.part | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 66ced5ff8ffa..81b018eaad15 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1435,6 +1435,8 @@ physical_plan statement ok set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; +# TODO: The selection vector partitioning should be used for the hash join. +# After fix https://github.com/apache/datafusion/issues/15382 query TT explain SELECT * FROM @@ -1474,10 +1476,6 @@ physical_plan 19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] 20)----------DataSourceExec: partitions=1, partition_sizes=[1] -# TODO: It should be: -# 2 2 2 2 -# 3 3 3 3 -# After fix https://github.com/apache/datafusion/issues/15382 query IIII SELECT * FROM (SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) From 35e540218762c362be59200aa9624741e1c8f804 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 02:02:56 +0800 Subject: [PATCH 16/37] fix typo --- datafusion/physical-plan/src/repartition/mod.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 544d76eb3209..ca212777feeb 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -206,7 +206,7 @@ enum BatchPartitionerState { }, } -pub static SELECTION_FILED_NAME: &str = "__selection"; +pub static SELECTION_FIELD_NAME: &str = "__selection"; impl BatchPartitioner { /// Create a new [`BatchPartitioner`] with the provided [`Partitioning`] @@ -367,7 +367,7 @@ impl BatchPartitioner { .map(Arc::clone) .collect::>(); fields.push(Arc::new(Field::new( - SELECTION_FILED_NAME, + SELECTION_FIELD_NAME, DataType::Boolean, false, ))); @@ -1308,7 +1308,7 @@ mod tests { .map(|x| { x.iter() .map(|x| { - x.column_by_name(SELECTION_FILED_NAME) + x.column_by_name(SELECTION_FIELD_NAME) .unwrap() .as_boolean() .iter() @@ -1321,7 +1321,7 @@ mod tests { let expected_schema = Arc::new(Schema::new(vec![ Field::new("c0", DataType::UInt32, false), - Field::new(SELECTION_FILED_NAME, DataType::Boolean, false), + Field::new(SELECTION_FIELD_NAME, DataType::Boolean, false), ])); assert_eq!(expected_schema, output_partitions[0][0].schema()); From b489378bc88ac634d85b2d22725195b081fcfaf2 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 02:15:10 +0800 Subject: [PATCH 17/37] address review comment --- datafusion/physical-plan/src/repartition/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index ca212777feeb..e374a1db3d10 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -387,8 +387,7 @@ impl BatchPartitioner { .map(|&hash| Some(hash == partition as u64)) .collect::(), ); - let mut columns = - batch.columns().iter().map(Arc::clone).collect::>(); + let mut columns = batch.columns().to_vec(); columns.push(selection_vector); let mut options = RecordBatchOptions::new(); options = options.with_row_count(Some(batch.num_rows())); From 0f9f398be39370b0ead8342fda9c2f21247e3aa5 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 02:34:36 +0800 Subject: [PATCH 18/37] fix compile and ehance doc --- .../core/tests/physical_optimizer/enforce_distribution.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 64aa7993242c..144e6ee95e5e 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -45,7 +45,7 @@ use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{HashPartitionMode, PhysicalExpr}; use datafusion_physical_expr::{ expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, }; @@ -281,7 +281,7 @@ fn aggregate_exec_with_alias( Arc::new( AggregateExec::try_new( - AggregateMode::FinalPartitioned, + AggregateMode::FinalPartitioned(HashPartitionMode::HashPartitioned), final_grouping, vec![], vec![], From 2b93d800c1e763003bd7097db5e7f9d8e98dbdde Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 02:38:36 +0800 Subject: [PATCH 19/37] rename config --- datafusion/common/src/config.rs | 4 ++-- datafusion/core/src/physical_planner.rs | 4 ++-- datafusion/physical-expr/src/partitioning.rs | 3 ++- datafusion/sqllogictest/test_files/aggregate.slt | 4 ++-- datafusion/sqllogictest/test_files/information_schema.slt | 4 ++-- datafusion/sqllogictest/test_files/join.slt.part | 4 ++-- docs/source/user-guide/configs.md | 2 +- 7 files changed, 13 insertions(+), 12 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index dcbd69a5bc41..acb6b998abf1 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -700,9 +700,9 @@ config_namespace! { /// HashJoin can work more efficiently than SortMergeJoin but consumes more memory pub prefer_hash_join: bool, default = true - /// When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec + /// When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for HashAggregate /// over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. - pub prefer_hash_selection_vector_partitioning: bool, default = false + pub prefer_hash_selection_vector_partitioning_agg: bool, default = false /// The maximum estimated size in bytes for one input side of a HashJoin /// will be collected into a single partition diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index a2f4c3ce8b43..737e3bc1b48e 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -744,7 +744,7 @@ impl DefaultPhysicalPlanner { let mode = if session_state .config_options() .optimizer - .prefer_hash_selection_vector_partitioning + .prefer_hash_selection_vector_partitioning_agg { HashPartitionMode::SelectionVector } else { @@ -816,7 +816,7 @@ impl DefaultPhysicalPlanner { if session_state .config_options() .optimizer - .prefer_hash_selection_vector_partitioning + .prefer_hash_selection_vector_partitioning_agg { Partitioning::HashSelectionVector(runtime_expr, *n) } else { diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 932c963f6613..ccec8d6e6ea2 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -304,9 +304,10 @@ impl Display for Distribution { } } +/// The mode of hash partitioning #[derive(Debug, Copy, Clone, PartialEq, Eq)] pub enum HashPartitionMode { - /// Hash partitioning with a selection vector + /// Hash partitioning with a selection vector. See [Partitioning::HashSelectionVector] SelectionVector, /// The default hash partitioning HashPartitioned, diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 909ce117e23c..d5634c109d75 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1120,7 +1120,7 @@ physical_plan 10)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; +set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = true; query TT explain select median(distinct c) from t; @@ -1149,7 +1149,7 @@ select count(distinct c) from t; 12 statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning = false; +set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = false; statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 6cbd6895a794..7d6164edf64a 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -295,7 +295,7 @@ datafusion.optimizer.max_passes 3 datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_existing_union false datafusion.optimizer.prefer_hash_join true -datafusion.optimizer.prefer_hash_selection_vector_partitioning false +datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg false datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 datafusion.optimizer.repartition_file_scans true @@ -406,7 +406,7 @@ datafusion.optimizer.max_passes 3 Number of times that the optimizer will attemp datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory -datafusion.optimizer.prefer_hash_selection_vector_partitioning false When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. +datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg false When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level datafusion.optimizer.repartition_file_min_size 10485760 Minimum total files size in bytes to perform file scan repartitioning. datafusion.optimizer.repartition_file_scans true When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 81b018eaad15..ebc3586a938d 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1433,7 +1433,7 @@ physical_plan 20)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning = true; +set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = true; # TODO: The selection vector partitioning should be used for the hash join. # After fix https://github.com/apache/datafusion/issues/15382 @@ -1489,7 +1489,7 @@ ON col1 = s.b # reset the config statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning = false; +set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = false; # reset the config statement ok diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 9aa3c94f1833..5f2233c8f9f3 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -114,7 +114,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | | datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.prefer_hash_selection_vector_partitioning | false | When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. | +| datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg | false | When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for HashAggregate over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. | | datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | From bad2e15a8c23263181dc0d40817ee8024faaabcf Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 02:40:38 +0800 Subject: [PATCH 20/37] fix fmt --- .../core/tests/physical_optimizer/enforce_distribution.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 144e6ee95e5e..3b395f66e58e 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -45,10 +45,10 @@ use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; -use datafusion_physical_expr::{HashPartitionMode, PhysicalExpr}; use datafusion_physical_expr::{ expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, }; +use datafusion_physical_expr::{HashPartitionMode, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::enforce_distribution::*; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; From bc7065365132b4d6f94697b8461370ada75820eb Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 03:00:43 +0800 Subject: [PATCH 21/37] fix test --- datafusion/core/tests/dataframe/mod.rs | 46 +++++++------- .../enforce_distribution.rs | 60 +++++++++---------- datafusion/core/tests/sql/explain_analyze.rs | 4 +- 3 files changed, 55 insertions(+), 55 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 1855a512048d..7f5840524f82 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2514,7 +2514,7 @@ async fn test_count_wildcard_on_sort() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), - @r###" + @r" +---------------+------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+------------------------------------------------------------------------------------------------------------+ @@ -2527,7 +2527,7 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] | | | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] | | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] | - | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] | + | | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | @@ -2535,29 +2535,29 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | DataSourceExec: partitions=1, partition_sizes=[1] | | | | +---------------+------------------------------------------------------------------------------------------------------------+ - "### + " ); assert_snapshot!( pretty_format_batches(&df_results).unwrap(), - @r###" - +---------------+--------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+--------------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) ASC NULLS LAST | - | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t1 projection=[b] | - | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | - | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | - | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+--------------------------------------------------------------------------------+ - "### + @r" + +---------------+--------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+--------------------------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) ASC NULLS LAST | + | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t1 projection=[b] | + | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | + | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | + | | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+--------------------------------------------------------------------------------------------+ + " ); Ok(()) } @@ -2870,7 +2870,7 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | | | DataSourceExec: partitions=1, partition_sizes=[1] | | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | @@ -2927,7 +2927,7 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | | | DataSourceExec: partitions=1, partition_sizes=[1] | | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[count(*)] | | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 3b395f66e58e..22905139653b 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -882,12 +882,12 @@ fn join_after_agg_alias() -> Result<()> { // Only two RepartitionExecs added let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a1@0 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a2@0 as a2], aggr=[]", " RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -936,12 +936,12 @@ fn hash_join_key_ordering() -> Result<()> { let expected = &[ "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b1@0 as b1, a1@1 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b, a@1 as a], aggr=[]", " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1694,14 +1694,14 @@ fn smj_join_key_ordering() -> Result<()> { " SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", " ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b1@0 as b1, a1@1 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", " SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", " ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b, a@1 as a], aggr=[]", " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1718,7 +1718,7 @@ fn smj_join_key_ordering() -> Result<()> { " CoalescePartitionsExec", " ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b1@0 as b1, a1@1 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1728,7 +1728,7 @@ fn smj_join_key_ordering() -> Result<()> { " SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[b@0 as b, a@1 as a], aggr=[]", " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1809,15 +1809,15 @@ fn union_to_interleave() -> Result<()> { // Only two RepartitionExecs added, no final RepartitionExec required let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a2@0 as a2], aggr=[]", " AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", " InterleaveExec", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a1@0 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a1@0 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1853,16 +1853,16 @@ fn union_not_to_interleave() -> Result<()> { // Only two RepartitionExecs added, no final RepartitionExec required let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a2@0 as a2], aggr=[]", " RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", " AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", " UnionExec", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a1@0 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a1@0 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1884,7 +1884,7 @@ fn added_repartition_to_single_partition() -> Result<()> { let plan = aggregate_exec_with_alias(parquet_exec(), alias); let expected = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1904,7 +1904,7 @@ fn repartition_deepest_node() -> Result<()> { let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " FilterExec: c@2 = 0", @@ -2002,7 +2002,7 @@ fn repartition_ignores_limit() -> Result<()> { ); let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -2426,7 +2426,7 @@ fn parallelization_single_partition() -> Result<()> { // Test: with parquet let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -2440,7 +2440,7 @@ fn parallelization_single_partition() -> Result<()> { // Test: with csv let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", @@ -2520,7 +2520,7 @@ fn parallelization_compressed_csv() -> Result<()> { ]; let expected_not_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -2528,7 +2528,7 @@ fn parallelization_compressed_csv() -> Result<()> { ]; let expected_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", @@ -2575,7 +2575,7 @@ fn parallelization_two_partitions() -> Result<()> { // Test: with parquet let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions @@ -2590,7 +2590,7 @@ fn parallelization_two_partitions() -> Result<()> { // Test: with csv let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions @@ -2614,7 +2614,7 @@ fn parallelization_two_partitions_into_four() -> Result<()> { // Test: with parquet let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions @@ -2629,7 +2629,7 @@ fn parallelization_two_partitions_into_four() -> Result<()> { // Test: with csv let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions @@ -2753,7 +2753,7 @@ fn parallelization_ignores_limit() -> Result<()> { // Test: with parquet let expected_parquet = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -2777,7 +2777,7 @@ fn parallelization_ignores_limit() -> Result<()> { // Test: with csv let expected_csv = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3388,7 +3388,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let test_config = TestConfig::default().with_query_execution_partitions(1); let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; @@ -3415,11 +3415,11 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let test_config = TestConfig::default().with_query_execution_partitions(4); let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", // Since hash requirements of this operator is satisfied. There shouldn't be // a hash repartition here " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index e8ef34c2afe7..b91f0edbfe21 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -61,7 +61,7 @@ async fn explain_analyze_baseline_metrics() { ); assert_metrics!( &formatted, - "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1]", + "AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1]", "metrics=[output_rows=5, elapsed_compute=" ); assert_metrics!( @@ -604,7 +604,7 @@ async fn test_physical_plan_display_indent() { "SortPreservingMergeExec: [the_min@2 DESC], fetch=10", " SortExec: TopK(fetch=10), expr=[the_min@2 DESC], preserve_partitioning=[true]", " ProjectionExec: expr=[c1@0 as c1, max(aggregate_test_100.c12)@1 as max(aggregate_test_100.c12), min(aggregate_test_100.c12)@2 as the_min]", - " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", + " AggregateExec: mode=FinalPartitioned(HashPartitioned), gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", From 764cce709b0c879f5a9f9e985e586b7ac83fb4f3 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 30 Mar 2025 03:04:02 +0800 Subject: [PATCH 22/37] fix sqllogictests --- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- datafusion/sqllogictest/test_files/join.slt.part | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 7d6164edf64a..7e0e1ef57628 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -406,7 +406,7 @@ datafusion.optimizer.max_passes 3 Number of times that the optimizer will attemp datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory -datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg false When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for RepartitionExec over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. +datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg false When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for HashAggregate over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level datafusion.optimizer.repartition_file_min_size 10485760 Minimum total files size in bytes to perform file scan repartitioning. datafusion.optimizer.repartition_file_scans true When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index ebc3586a938d..b29a63987de1 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1476,7 +1476,7 @@ physical_plan 19)--------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] 20)----------DataSourceExec: partitions=1, partition_sizes=[1] -query IIII +query IIII rowsort SELECT * FROM (SELECT x+1 AS col0, y+1 AS col1 FROM PAIRS WHERE x == y) JOIN f @@ -1484,8 +1484,8 @@ ON col0 = f.a JOIN s ON col1 = s.b ---- -3 3 3 3 2 2 2 2 +3 3 3 3 # reset the config statement ok From c01119cbb813bb17655e6c1a09e2a72bb3e0085b Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 13 Apr 2025 23:05:24 +0800 Subject: [PATCH 23/37] fix fmt --- datafusion/proto/src/physical_plan/mod.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 8cf37bf30115..5fe98f9f2540 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -935,9 +935,9 @@ impl protobuf::PhysicalPlanNode { let agg_mode: AggregateMode = match mode { protobuf::AggregateMode::Partial => AggregateMode::Partial, protobuf::AggregateMode::Final => AggregateMode::Final, - protobuf::AggregateMode::FinalPartitioned => AggregateMode::FinalPartitioned( - HashPartitionMode::HashPartitioned, - ), + protobuf::AggregateMode::FinalPartitioned => { + AggregateMode::FinalPartitioned(HashPartitionMode::HashPartitioned) + } protobuf::AggregateMode::Single => AggregateMode::Single, protobuf::AggregateMode::SinglePartitioned => { AggregateMode::SinglePartitioned @@ -2152,7 +2152,9 @@ impl protobuf::PhysicalPlanNode { let agg_mode = match exec.mode() { AggregateMode::Partial => protobuf::AggregateMode::Partial, AggregateMode::Final => protobuf::AggregateMode::Final, - AggregateMode::FinalPartitioned(_) => protobuf::AggregateMode::FinalPartitioned, + AggregateMode::FinalPartitioned(_) => { + protobuf::AggregateMode::FinalPartitioned + } AggregateMode::Single => protobuf::AggregateMode::Single, AggregateMode::SinglePartitioned => { protobuf::AggregateMode::SinglePartitioned From 057660fb7584cb7dcb85149f4d2c0b60d3fc2881 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 19 Apr 2025 23:20:11 +0800 Subject: [PATCH 24/37] optimze repartition --- .../physical-plan/src/repartition/mod.rs | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index e374a1db3d10..131d6cad6ee6 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,9 +40,9 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions}; -use arrow::compute::take_arrays; -use arrow::datatypes::{SchemaRef, UInt32Type}; +use arrow::array::{BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array}; +use arrow::compute::{kernels, take_arrays}; +use arrow::datatypes::{BooleanType, SchemaRef, UInt32Type, UInt64Type}; use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; @@ -344,22 +344,20 @@ impl BatchPartitioner { random_state, exprs, num_partitions, - hash_buffer, + .. } => { + let mut hash_buffer = vec![]; let timer = self.timer.timer(); let arrays = exprs .iter() .map(|expr| expr.evaluate(&batch)?.into_array(batch.num_rows())) .collect::>>()?; - hash_buffer.clear(); hash_buffer.resize(batch.num_rows(), 0); - create_hashes(&arrays, random_state, hash_buffer)?; + create_hashes(&arrays, random_state, &mut hash_buffer)?; - let hash_vector = hash_buffer - .iter() - .map(|hash| *hash % *num_partitions as u64) - .collect::>(); + let hash_vector = UInt64Array::from(hash_buffer).unary_mut(|a| a % *num_partitions as u64).unwrap(); + let mut fields = batch .schema() .fields() @@ -381,12 +379,11 @@ impl BatchPartitioner { let it = (0..*num_partitions).map(move |partition| { // Tracking time required for repartitioned batches construction let _timer = partitioner_timer.timer(); - let selection_vector = Arc::new( - hash_vector - .iter() - .map(|&hash| Some(hash == partition as u64)) - .collect::(), - ); + let selection_array = arrow_ord::cmp::eq( + &hash_vector, + &UInt64Array::from(vec![partition as u64; batch.num_rows()]), + ).unwrap(); + let selection_vector = Arc::new(selection_array); let mut columns = batch.columns().to_vec(); columns.push(selection_vector); let mut options = RecordBatchOptions::new(); From 40caf0c05ae9254887a5358914f0bb73d29c25be Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 20 Apr 2025 14:42:26 +0800 Subject: [PATCH 25/37] fix fmt --- datafusion/physical-plan/src/repartition/mod.rs | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 131d6cad6ee6..3ebfdd10d712 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,7 +40,9 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array}; +use arrow::array::{ + BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array, +}; use arrow::compute::{kernels, take_arrays}; use arrow::datatypes::{BooleanType, SchemaRef, UInt32Type, UInt64Type}; use arrow_schema::{DataType, Field}; @@ -356,8 +358,10 @@ impl BatchPartitioner { hash_buffer.resize(batch.num_rows(), 0); create_hashes(&arrays, random_state, &mut hash_buffer)?; - let hash_vector = UInt64Array::from(hash_buffer).unary_mut(|a| a % *num_partitions as u64).unwrap(); - + let hash_vector = UInt64Array::from(hash_buffer) + .unary_mut(|a| a % *num_partitions as u64) + .unwrap(); + let mut fields = batch .schema() .fields() @@ -382,7 +386,8 @@ impl BatchPartitioner { let selection_array = arrow_ord::cmp::eq( &hash_vector, &UInt64Array::from(vec![partition as u64; batch.num_rows()]), - ).unwrap(); + ) + .unwrap(); let selection_vector = Arc::new(selection_array); let mut columns = batch.columns().to_vec(); columns.push(selection_vector); From 84ebdc9f3197e747e7b2825194f7175a50b8e8c6 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 20 Apr 2025 15:05:30 +0800 Subject: [PATCH 26/37] fix clippy and fmt --- datafusion/physical-plan/src/repartition/mod.rs | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 3ebfdd10d712..2614f3c5401d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,11 +40,9 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{ - BooleanArray, PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array, -}; -use arrow::compute::{kernels, take_arrays}; -use arrow::datatypes::{BooleanType, SchemaRef, UInt32Type, UInt64Type}; +use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array}; +use arrow::compute::take_arrays; +use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; @@ -204,7 +202,6 @@ enum BatchPartitionerState { random_state: ahash::RandomState, exprs: Vec>, num_partitions: usize, - hash_buffer: Vec, }, } @@ -235,7 +232,6 @@ impl BatchPartitioner { num_partitions, // Use fixed random hash random_state: ahash::RandomState::with_seeds(0, 0, 0, 0), - hash_buffer: vec![], } } other => return not_impl_err!("Unsupported repartitioning scheme {other:?}"), @@ -346,7 +342,6 @@ impl BatchPartitioner { random_state, exprs, num_partitions, - .. } => { let mut hash_buffer = vec![]; let timer = self.timer.timer(); From ebdab1f5531439e99290a50e3f603c0919b8a809 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Mon, 21 Apr 2025 21:31:40 +0800 Subject: [PATCH 27/37] use scalar instead array --- datafusion/physical-plan/src/repartition/mod.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 2614f3c5401d..3a183199b4f9 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,14 +40,16 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array}; +use arrow::array::{ + PrimitiveArray, RecordBatch, RecordBatchOptions, Scalar, UInt64Array, +}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; -use datafusion_common::HashMap; use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common::{HashMap, ScalarValue}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; @@ -378,11 +380,9 @@ impl BatchPartitioner { let it = (0..*num_partitions).map(move |partition| { // Tracking time required for repartitioned batches construction let _timer = partitioner_timer.timer(); - let selection_array = arrow_ord::cmp::eq( - &hash_vector, - &UInt64Array::from(vec![partition as u64; batch.num_rows()]), - ) - .unwrap(); + let partition_scalar = UInt64Array::new_scalar(partition as u64); + let selection_array = + arrow_ord::cmp::eq(&hash_vector, &partition_scalar).unwrap(); let selection_vector = Arc::new(selection_array); let mut columns = batch.columns().to_vec(); columns.push(selection_vector); From 401cb9c1d012acadc2a04ea8522ed3d6b5de6a78 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Tue, 29 Apr 2025 23:23:16 +0800 Subject: [PATCH 28/37] fix clippy --- datafusion/physical-plan/src/repartition/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 3a183199b4f9..552655a34940 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -41,7 +41,7 @@ use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; use arrow::array::{ - PrimitiveArray, RecordBatch, RecordBatchOptions, Scalar, UInt64Array, + PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array, }; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; @@ -49,7 +49,7 @@ use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; use datafusion_common::{not_impl_err, DataFusionError, Result}; -use datafusion_common::{HashMap, ScalarValue}; +use datafusion_common::HashMap; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; From 8d7d038eaf47d16f052def8f64427883d94598ba Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Tue, 29 Apr 2025 23:34:00 +0800 Subject: [PATCH 29/37] fix fmt --- datafusion/physical-plan/src/repartition/mod.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 552655a34940..8a6622c0f6a7 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,16 +40,14 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{ - PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array, -}; +use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions, UInt64Array}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::transpose; -use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common::HashMap; +use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; From e3c7021f122e58dd193f94b3e327acc5f8725f67 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 4 May 2025 22:23:25 +0800 Subject: [PATCH 30/37] rename selection_vector to selection_bitmap --- datafusion/common/src/config.rs | 6 ++-- datafusion/core/src/physical_planner.rs | 8 ++--- datafusion/physical-expr/src/partitioning.rs | 20 +++++------ .../physical-plan/src/repartition/mod.rs | 16 ++++----- datafusion/proto/proto/datafusion.proto | 6 ++-- datafusion/proto/src/generated/pbjson.rs | 34 +++++++++---------- datafusion/proto/src/generated/prost.rs | 6 ++-- .../proto/src/physical_plan/from_proto.rs | 10 +++--- .../proto/src/physical_plan/to_proto.rs | 6 ++-- .../sqllogictest/test_files/aggregate.slt | 8 ++--- .../test_files/information_schema.slt | 4 +-- .../sqllogictest/test_files/join.slt.part | 4 +-- 12 files changed, 64 insertions(+), 64 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index acb6b998abf1..617408ff08bd 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -700,9 +700,9 @@ config_namespace! { /// HashJoin can work more efficiently than SortMergeJoin but consumes more memory pub prefer_hash_join: bool, default = true - /// When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for HashAggregate - /// over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. - pub prefer_hash_selection_vector_partitioning_agg: bool, default = false + /// When set to true, the physical plan optimizer will prefer HashSelectionBitmapPartitioning for HashAggregate + /// over HashPartitioning. HashSelectionBitmapPartitioning can work without data copying. + pub prefer_hash_selection_bitmap_partitioning_agg: bool, default = false /// The maximum estimated size in bytes for one input side of a HashJoin /// will be collected into a single partition diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 737e3bc1b48e..e12cbf35559b 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -744,9 +744,9 @@ impl DefaultPhysicalPlanner { let mode = if session_state .config_options() .optimizer - .prefer_hash_selection_vector_partitioning_agg + .prefer_hash_selection_bitmap_partitioning_agg { - HashPartitionMode::SelectionVector + HashPartitionMode::SelectionBitmap } else { HashPartitionMode::HashPartitioned }; @@ -816,9 +816,9 @@ impl DefaultPhysicalPlanner { if session_state .config_options() .optimizer - .prefer_hash_selection_vector_partitioning_agg + .prefer_hash_selection_bitmap_partitioning_agg { - Partitioning::HashSelectionVector(runtime_expr, *n) + Partitioning::HashSelectionBitmap(runtime_expr, *n) } else { Partitioning::Hash(runtime_expr, *n) } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index ccec8d6e6ea2..a1f3f165ade4 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -123,7 +123,7 @@ pub enum Partitioning { /// The column is a boolean column called `__selection` that is used to filter out rows /// that should not be included in the partition. `true` means the row should be included /// and `false` means the row should be excluded. - HashSelectionVector(Vec>, usize), + HashSelectionBitmap(Vec>, usize), /// Unknown partitioning scheme with a known number of partitions UnknownPartitioning(usize), } @@ -140,13 +140,13 @@ impl Display for Partitioning { .join(", "); write!(f, "Hash([{phy_exprs_str}], {size})") } - Partitioning::HashSelectionVector(phy_exprs, size) => { + Partitioning::HashSelectionBitmap(phy_exprs, size) => { let phy_exprs_str = phy_exprs .iter() .map(|e| format!("{e}")) .collect::>() .join(", "); - write!(f, "HashSelectionVector([{phy_exprs_str}], {size})") + write!(f, "HashSelectionBitmap([{phy_exprs_str}], {size})") } Partitioning::UnknownPartitioning(size) => { write!(f, "UnknownPartitioning({size})") @@ -161,7 +161,7 @@ impl Partitioning { match self { RoundRobinBatch(n) | Hash(_, n) - | HashSelectionVector(_, n) + | HashSelectionBitmap(_, n) | UnknownPartitioning(n) => *n, } } @@ -184,7 +184,7 @@ impl Partitioning { // and hash functions in the system are the same. In future if we plan to support storage partition-wise joins, // then we need to have the partition count and hash functions validation. Partitioning::Hash(partition_exprs, _) - | Partitioning::HashSelectionVector(partition_exprs, _) => { + | Partitioning::HashSelectionBitmap(partition_exprs, _) => { let fast_match = physical_exprs_equal(required_exprs, partition_exprs); // If the required exprs do not match, need to leverage the eq_properties provided by the child @@ -280,8 +280,8 @@ impl Distribution { Distribution::HashPartitioned(expr, HashPartitionMode::HashPartitioned) => { Partitioning::Hash(expr, partition_count) } - Distribution::HashPartitioned(expr, HashPartitionMode::SelectionVector) => { - Partitioning::HashSelectionVector(expr, partition_count) + Distribution::HashPartitioned(expr, HashPartitionMode::SelectionBitmap) => { + Partitioning::HashSelectionBitmap(expr, partition_count) } } } @@ -307,8 +307,8 @@ impl Display for Distribution { /// The mode of hash partitioning #[derive(Debug, Copy, Clone, PartialEq, Eq)] pub enum HashPartitionMode { - /// Hash partitioning with a selection vector. See [Partitioning::HashSelectionVector] - SelectionVector, + /// Hash partitioning with a selection vector. See [Partitioning::HashSelectionBitmap] + SelectionBitmap, /// The default hash partitioning HashPartitioned, } @@ -316,7 +316,7 @@ pub enum HashPartitionMode { impl Display for HashPartitionMode { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { - HashPartitionMode::SelectionVector => write!(f, "SelectionVector"), + HashPartitionMode::SelectionBitmap => write!(f, "SelectionBitmap"), HashPartitionMode::HashPartitioned => write!(f, "HashPartitioned"), } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 8a6622c0f6a7..7d33bc1ce73b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -198,7 +198,7 @@ enum BatchPartitionerState { num_partitions: usize, next_idx: usize, }, - HashSelectionVector { + HashSelectionBitmap { random_state: ahash::RandomState, exprs: Vec>, num_partitions: usize, @@ -226,8 +226,8 @@ impl BatchPartitioner { random_state: ahash::RandomState::with_seeds(0, 0, 0, 0), hash_buffer: vec![], }, - Partitioning::HashSelectionVector(exprs, num_partitions) => { - BatchPartitionerState::HashSelectionVector { + Partitioning::HashSelectionBitmap(exprs, num_partitions) => { + BatchPartitionerState::HashSelectionBitmap { exprs, num_partitions, // Use fixed random hash @@ -338,7 +338,7 @@ impl BatchPartitioner { Box::new(it) } - BatchPartitionerState::HashSelectionVector { + BatchPartitionerState::HashSelectionBitmap { random_state, exprs, num_partitions, @@ -381,9 +381,9 @@ impl BatchPartitioner { let partition_scalar = UInt64Array::new_scalar(partition as u64); let selection_array = arrow_ord::cmp::eq(&hash_vector, &partition_scalar).unwrap(); - let selection_vector = Arc::new(selection_array); + let selection_bitmap = Arc::new(selection_array); let mut columns = batch.columns().to_vec(); - columns.push(selection_vector); + columns.push(selection_bitmap); let mut options = RecordBatchOptions::new(); options = options.with_row_count(Some(batch.num_rows())); let batch = RecordBatch::try_new_with_options( @@ -408,7 +408,7 @@ impl BatchPartitioner { match self.state { BatchPartitionerState::RoundRobin { num_partitions, .. } => num_partitions, BatchPartitionerState::Hash { num_partitions, .. } => num_partitions, - BatchPartitionerState::HashSelectionVector { num_partitions, .. } => { + BatchPartitionerState::HashSelectionBitmap { num_partitions, .. } => { num_partitions } } @@ -1293,7 +1293,7 @@ mod tests { let output_partitions = repartition( &schema, partitions, - Partitioning::HashSelectionVector(vec![col("c0", &schema)?], 8), + Partitioning::HashSelectionBitmap(vec![col("c0", &schema)?], 8), ) .await?; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 207141a7cf64..0c25b3ff75fb 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1225,7 +1225,7 @@ message PhysicalHashRepartition { uint64 partition_count = 2; } -message PhysicalHashSelectionVectorRepartition { +message PhysicalHashSelectionBitmapRepartition { repeated PhysicalExprNode hash_expr = 1; uint64 partition_count = 2; } @@ -1236,7 +1236,7 @@ message RepartitionExecNode{ // uint64 round_robin = 2; // PhysicalHashRepartition hash = 3; // uint64 unknown = 4; - // PhysicalHashSelectionVectorRepartition hash_selection_vector = 6; + // PhysicalHashSelectionBitmapRepartition hash_selection_bitmap = 6; // } Partitioning partitioning = 5; } @@ -1246,7 +1246,7 @@ message Partitioning { uint64 round_robin = 1; PhysicalHashRepartition hash = 2; uint64 unknown = 3; - PhysicalHashSelectionVectorRepartition hash_selection_vector = 4; + PhysicalHashSelectionBitmapRepartition hash_selection_bitmap = 4; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 0b858f4d23db..b8c4052c3204 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -13403,8 +13403,8 @@ impl serde::Serialize for Partitioning { #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("unknown", ToString::to_string(&v).as_str())?; } - partitioning::PartitionMethod::HashSelectionVector(v) => { - struct_ser.serialize_field("hashSelectionVector", v)?; + partitioning::PartitionMethod::HashSelectionBitmap(v) => { + struct_ser.serialize_field("hashSelectionBitmap", v)?; } } } @@ -13422,8 +13422,8 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin", "hash", "unknown", - "hash_selection_vector", - "hashSelectionVector", + "hash_selection_bitmap", + "hashSelectionBitmap", ]; #[allow(clippy::enum_variant_names)] @@ -13431,7 +13431,7 @@ impl<'de> serde::Deserialize<'de> for Partitioning { RoundRobin, Hash, Unknown, - HashSelectionVector, + HashSelectionBitmap, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -13456,7 +13456,7 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin" | "round_robin" => Ok(GeneratedField::RoundRobin), "hash" => Ok(GeneratedField::Hash), "unknown" => Ok(GeneratedField::Unknown), - "hashSelectionVector" | "hash_selection_vector" => Ok(GeneratedField::HashSelectionVector), + "hashSelectionBitmap" | "hash_selection_bitmap" => Ok(GeneratedField::HashSelectionBitmap), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -13498,11 +13498,11 @@ impl<'de> serde::Deserialize<'de> for Partitioning { } partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| partitioning::PartitionMethod::Unknown(x.0)); } - GeneratedField::HashSelectionVector => { + GeneratedField::HashSelectionBitmap => { if partition_method__.is_some() { - return Err(serde::de::Error::duplicate_field("hashSelectionVector")); + return Err(serde::de::Error::duplicate_field("hashSelectionBitmap")); } - partition_method__ = map_.next_value::<::std::option::Option<_>>()?.map(partitioning::PartitionMethod::HashSelectionVector) + partition_method__ = map_.next_value::<::std::option::Option<_>>()?.map(partitioning::PartitionMethod::HashSelectionBitmap) ; } } @@ -15074,7 +15074,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalHashRepartition { deserializer.deserialize_struct("datafusion.PhysicalHashRepartition", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for PhysicalHashSelectionVectorRepartition { +impl serde::Serialize for PhysicalHashSelectionBitmapRepartition { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -15088,7 +15088,7 @@ impl serde::Serialize for PhysicalHashSelectionVectorRepartition { if self.partition_count != 0 { len += 1; } - let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalHashSelectionVectorRepartition", len)?; + let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalHashSelectionBitmapRepartition", len)?; if !self.hash_expr.is_empty() { struct_ser.serialize_field("hashExpr", &self.hash_expr)?; } @@ -15100,7 +15100,7 @@ impl serde::Serialize for PhysicalHashSelectionVectorRepartition { struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for PhysicalHashSelectionVectorRepartition { +impl<'de> serde::Deserialize<'de> for PhysicalHashSelectionBitmapRepartition { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where @@ -15149,13 +15149,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalHashSelectionVectorRepartition { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = PhysicalHashSelectionVectorRepartition; + type Value = PhysicalHashSelectionBitmapRepartition; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.PhysicalHashSelectionVectorRepartition") + formatter.write_str("struct datafusion.PhysicalHashSelectionBitmapRepartition") } - fn visit_map(self, mut map_: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -15179,13 +15179,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalHashSelectionVectorRepartition { } } } - Ok(PhysicalHashSelectionVectorRepartition { + Ok(PhysicalHashSelectionBitmapRepartition { hash_expr: hash_expr__.unwrap_or_default(), partition_count: partition_count__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("datafusion.PhysicalHashSelectionVectorRepartition", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("datafusion.PhysicalHashSelectionBitmapRepartition", FIELDS, GeneratedVisitor) } } impl serde::Serialize for PhysicalInListNode { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 4189aed9113f..4024dd87efb5 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1835,7 +1835,7 @@ pub struct PhysicalHashRepartition { pub partition_count: u64, } #[derive(Clone, PartialEq, ::prost::Message)] -pub struct PhysicalHashSelectionVectorRepartition { +pub struct PhysicalHashSelectionBitmapRepartition { #[prost(message, repeated, tag = "1")] pub hash_expr: ::prost::alloc::vec::Vec, #[prost(uint64, tag = "2")] @@ -1849,7 +1849,7 @@ pub struct RepartitionExecNode { /// uint64 round_robin = 2; /// PhysicalHashRepartition hash = 3; /// uint64 unknown = 4; - /// PhysicalHashSelectionVectorRepartition hash_selection_vector = 6; + /// PhysicalHashSelectionBitmapRepartition hash_selection_bitmap = 6; /// } #[prost(message, optional, tag = "5")] pub partitioning: ::core::option::Option, @@ -1870,7 +1870,7 @@ pub mod partitioning { #[prost(uint64, tag = "3")] Unknown(u64), #[prost(message, tag = "4")] - HashSelectionVector(super::PhysicalHashSelectionVectorRepartition), + HashSelectionBitmap(super::PhysicalHashSelectionBitmapRepartition), } } #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 5c7dfdbcd9b0..71527df273c2 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -440,8 +440,8 @@ pub fn parse_protobuf_hash_partitioning( } } -pub fn parse_protobuf_hash_selection_vector_partitioning( - partitioning: Option<&protobuf::PhysicalHashSelectionVectorRepartition>, +pub fn parse_protobuf_hash_selection_bitmap_partitioning( + partitioning: Option<&protobuf::PhysicalHashSelectionBitmapRepartition>, registry: &dyn FunctionRegistry, input_schema: &Schema, codec: &dyn PhysicalExtensionCodec, @@ -455,7 +455,7 @@ pub fn parse_protobuf_hash_selection_vector_partitioning( codec, )?; - Ok(Some(Partitioning::HashSelectionVector( + Ok(Some(Partitioning::HashSelectionBitmap( expr, hash_part.partition_count.try_into().unwrap(), ))) @@ -485,9 +485,9 @@ pub fn parse_protobuf_partitioning( codec, ) } - Some(protobuf::partitioning::PartitionMethod::HashSelectionVector( + Some(protobuf::partitioning::PartitionMethod::HashSelectionBitmap( hash_repartition, - )) => parse_protobuf_hash_selection_vector_partitioning( + )) => parse_protobuf_hash_selection_bitmap_partitioning( Some(hash_repartition), registry, input_schema, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 86df4466549e..7aff3b9e2e35 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -413,12 +413,12 @@ pub fn serialize_partitioning( )), } } - Partitioning::HashSelectionVector(exprs, partition_count) => { + Partitioning::HashSelectionBitmap(exprs, partition_count) => { let serialized_exprs = serialize_physical_exprs(exprs, codec)?; protobuf::Partitioning { partition_method: Some( - protobuf::partitioning::PartitionMethod::HashSelectionVector( - protobuf::PhysicalHashSelectionVectorRepartition { + protobuf::partitioning::PartitionMethod::HashSelectionBitmap( + protobuf::PhysicalHashSelectionBitmapRepartition { hash_expr: serialized_exprs, partition_count: *partition_count as u64, }, diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index d5634c109d75..9eb03684d5a7 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1120,7 +1120,7 @@ physical_plan 10)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = true; +set datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg = true; query TT explain select median(distinct c) from t; @@ -1135,9 +1135,9 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] -05)--------AggregateExec: mode=FinalPartitioned(SelectionVector), gby=[alias1@0 as alias1], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned(SelectionBitmap), gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=HashSelectionVector([alias1@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=HashSelectionBitmap([alias1@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] 10)------------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1149,7 +1149,7 @@ select count(distinct c) from t; 12 statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = false; +set datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg = false; statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 7e0e1ef57628..d2d8e6c9e3b4 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -295,7 +295,7 @@ datafusion.optimizer.max_passes 3 datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_existing_union false datafusion.optimizer.prefer_hash_join true -datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg false +datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg false datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 datafusion.optimizer.repartition_file_scans true @@ -406,7 +406,7 @@ datafusion.optimizer.max_passes 3 Number of times that the optimizer will attemp datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory -datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg false When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for HashAggregate over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. +datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg false When set to true, the physical plan optimizer will prefer HashSelectionBitmapPartitioning for HashAggregate over HashPartitioning. HashSelectionBitmapPartitioning can work without data copying. datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level datafusion.optimizer.repartition_file_min_size 10485760 Minimum total files size in bytes to perform file scan repartitioning. datafusion.optimizer.repartition_file_scans true When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index b29a63987de1..3c9f7f69b27e 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1433,7 +1433,7 @@ physical_plan 20)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = true; +set datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg = true; # TODO: The selection vector partitioning should be used for the hash join. # After fix https://github.com/apache/datafusion/issues/15382 @@ -1489,7 +1489,7 @@ ON col1 = s.b # reset the config statement ok -set datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg = false; +set datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg = false; # reset the config statement ok From c7e4ac4637434ca5803725cca2e6c5673cfeba38 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 4 May 2025 23:00:10 +0800 Subject: [PATCH 31/37] fix sqllogictest --- .../sqllogictest/test_files/explain_tree.slt | 94 ++----------------- 1 file changed, 8 insertions(+), 86 deletions(-) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index ee450a821632..e94e3b9da491 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -208,46 +208,6 @@ physical_plan 08)│ │ 09)│ mode: │ 10)│ FinalPartitioned │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ CoalesceBatchesExec │ -14)│ -------------------- │ -15)│ target_batch_size: │ -16)│ 8192 │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ RepartitionExec │ -20)│ -------------------- │ -21)│ partition_count(in->out): │ -22)│ 4 -> 4 │ -23)│ │ -24)│ partitioning_scheme: │ -25)│ Hash([string_col@0], 4) │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ AggregateExec │ -29)│ -------------------- │ -30)│ aggr: │ -31)│ sum(table1.bigint_col) │ -32)│ │ -33)│ group_by: string_col │ -34)│ mode: Partial │ -35)└─────────────┬─────────────┘ -36)┌─────────────┴─────────────┐ -37)│ RepartitionExec │ -38)│ -------------------- │ -39)│ partition_count(in->out): │ -40)│ 1 -> 4 │ -41)│ │ -42)│ partitioning_scheme: │ -43)│ RoundRobinBatch(4) │ -44)└─────────────┬─────────────┘ -45)┌─────────────┴─────────────┐ -46)│ DataSourceExec │ -47)│ -------------------- │ -48)│ files: 1 │ -49)│ format: csv │ -50)└───────────────────────────┘ 11)│ (HashPartitioned │ 12)│ ) │ 13)└─────────────┬─────────────┘ @@ -260,8 +220,8 @@ physical_plan 20)┌─────────────┴─────────────┐ 21)│ RepartitionExec │ 22)│ -------------------- │ -23)│ input_partition_count: │ -24)│ 4 │ +23)│ partition_count(in->out): │ +24)│ 4 -> 4 │ 25)│ │ 26)│ partitioning_scheme: │ 27)│ Hash([string_col@0], 4) │ @@ -278,8 +238,8 @@ physical_plan 38)┌─────────────┴─────────────┐ 39)│ RepartitionExec │ 40)│ -------------------- │ -41)│ input_partition_count: │ -42)│ 1 │ +41)│ partition_count(in->out): │ +42)│ 1 -> 4 │ 43)│ │ 44)│ partitioning_scheme: │ 45)│ RoundRobinBatch(4) │ @@ -1537,44 +1497,6 @@ physical_plan 23)│ ││ │ 24)│ mode: ││ mode: │ 25)│ FinalPartitioned ││ FinalPartitioned │ -26)└─────────────┬─────────────┘└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -28)│ CoalesceBatchesExec ││ CoalesceBatchesExec │ -29)│ -------------------- ││ -------------------- │ -30)│ target_batch_size: ││ target_batch_size: │ -31)│ 8192 ││ 8192 │ -32)└─────────────┬─────────────┘└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -34)│ RepartitionExec ││ RepartitionExec │ -35)│ -------------------- ││ -------------------- │ -36)│ partition_count(in->out): ││ partition_count(in->out): │ -37)│ 4 -> 4 ││ 4 -> 4 │ -38)│ ││ │ -39)│ partitioning_scheme: ││ partitioning_scheme: │ -40)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ -41)└─────────────┬─────────────┘└─────────────┬─────────────┘ -42)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -43)│ RepartitionExec ││ RepartitionExec │ -44)│ -------------------- ││ -------------------- │ -45)│ partition_count(in->out): ││ partition_count(in->out): │ -46)│ 1 -> 4 ││ 1 -> 4 │ -47)│ ││ │ -48)│ partitioning_scheme: ││ partitioning_scheme: │ -49)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ -50)└─────────────┬─────────────┘└─────────────┬─────────────┘ -51)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -52)│ AggregateExec ││ AggregateExec │ -53)│ -------------------- ││ -------------------- │ -54)│ group_by: name ││ group_by: name │ -55)│ mode: Partial ││ mode: Partial │ -56)└─────────────┬─────────────┘└─────────────┬─────────────┘ -57)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -58)│ DataSourceExec ││ DataSourceExec │ -59)│ -------------------- ││ -------------------- │ -60)│ bytes: 1320 ││ bytes: 1312 │ -61)│ format: memory ││ format: memory │ -62)│ rows: 1 ││ rows: 1 │ -63)└───────────────────────────┘└───────────────────────────┘ 26)│ (HashPartitioned ││ (HashPartitioned │ 27)│ ) ││ ) │ 28)└─────────────┬─────────────┘└─────────────┬─────────────┘ @@ -1587,8 +1509,8 @@ physical_plan 35)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 36)│ RepartitionExec ││ RepartitionExec │ 37)│ -------------------- ││ -------------------- │ -38)│ input_partition_count: ││ input_partition_count: │ -39)│ 4 ││ 4 │ +38)│ partition_count(in->out): ││ partition_count(in->out): │ +39)│ 4 -> 4 ││ 4 -> 4 │ 40)│ ││ │ 41)│ partitioning_scheme: ││ partitioning_scheme: │ 42)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ @@ -1596,8 +1518,8 @@ physical_plan 44)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 45)│ RepartitionExec ││ RepartitionExec │ 46)│ -------------------- ││ -------------------- │ -47)│ input_partition_count: ││ input_partition_count: │ -48)│ 1 ││ 1 │ +47)│ partition_count(in->out): ││ partition_count(in->out): │ +48)│ 1 -> 4 ││ 1 -> 4 │ 49)│ ││ │ 50)│ partitioning_scheme: ││ partitioning_scheme: │ 51)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ From 5c46c6b11a1b4c1552d3e6b8033a2527c87b5308 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 7 May 2025 22:26:03 +0800 Subject: [PATCH 32/37] add sanity check for filtered batch --- .../physical-optimizer/src/sanity_checker.rs | 22 +++++++++++++++++++ .../physical-plan/src/execution_plan.rs | 14 ++++++++++++ 2 files changed, 36 insertions(+) diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index 8edbb0f09114..efb6e9254b52 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -43,6 +43,7 @@ use itertools::izip; /// are not satisfied by their children. /// 2. Plans that use pipeline-breaking operators on infinite input(s), /// it is impossible to execute such queries (they will never generate output nor finish) +/// 3. Plans that produce filtered batches but their children do not consume them. #[derive(Default, Debug)] pub struct SanityCheckPlan {} @@ -104,6 +105,26 @@ pub fn check_finiteness_requirements( } } +/// This function checks if the plan produces filtered batches and if its +/// children consume them. If not, it returns an error. +pub fn check_filtered_batch_handle( + plan: Arc, +) -> Result>> { + if plan.output_filtered_batches() { + for child in plan.children() { + if child.requried_filtered_input() { + return plan_err!( + "Plan {:?} prodouces filtered batches but its child {:?} won't consume them", + plan, + child + ); + } + } + } + + Ok(Transformed::no(plan)) +} + /// This function returns whether a given symmetric hash join is amenable to /// data pruning. For this to be possible, it needs to have a filter where /// all involved [`PhysicalExpr`]s, [`Operator`]s and data types support @@ -129,6 +150,7 @@ pub fn check_plan_sanity( optimizer_options: &OptimizerOptions, ) -> Result>> { check_finiteness_requirements(Arc::clone(&plan), optimizer_options)?; + check_filtered_batch_handle(Arc::clone(&plan))?; for ((idx, child), sort_req, dist_req) in izip!( plan.children().into_iter().enumerate(), diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index b81b3c8beeac..9986d424fddd 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -220,6 +220,20 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(None) } + /// Returns `true` if this `ExecutionPlan` require the input batch with the + /// [`repartition::SELECTION_FIELD_NAME`] column which is a bitmap to indicate + /// which rows are selected for the current partition. + fn requried_filtered_input(&self) -> bool { + false + } + + /// Returns `true` if this `ExecutionPlan` will produce a filtered output with the + /// [`repartition::SELECTION_FIELD_NAME`] column which is a bitmap to indicate + /// which rows are selected for the current partition. + fn output_filtered_batches(&self) -> bool { + false + } + /// Begin execution of `partition`, returning a [`Stream`] of /// [`RecordBatch`]es. /// From be34338700966c466c40c62c04389cfa2e45f0e8 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 7 May 2025 23:07:07 +0800 Subject: [PATCH 33/37] fix doc --- datafusion/physical-plan/src/execution_plan.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 9986d424fddd..a4faf4ec0beb 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -221,14 +221,14 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } /// Returns `true` if this `ExecutionPlan` require the input batch with the - /// [`repartition::SELECTION_FIELD_NAME`] column which is a bitmap to indicate + /// [`crate::repartition::SELECTION_FIELD_NAME`] column which is a bitmap to indicate /// which rows are selected for the current partition. fn requried_filtered_input(&self) -> bool { false } /// Returns `true` if this `ExecutionPlan` will produce a filtered output with the - /// [`repartition::SELECTION_FIELD_NAME`] column which is a bitmap to indicate + /// [`crate::repartition::SELECTION_FIELD_NAME`] column which is a bitmap to indicate /// which rows are selected for the current partition. fn output_filtered_batches(&self) -> bool { false From d11f0ccd04370d5edeb0875e606787fc6480f452 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 8 May 2025 00:38:13 +0800 Subject: [PATCH 34/37] fix sanity check and add test --- .../physical_optimizer/sanity_checker.rs | 39 ++++++++++++++++++- .../tests/physical_optimizer/test_utils.rs | 29 +++++++++++++- .../physical-optimizer/src/sanity_checker.rs | 12 +++--- .../physical-plan/src/aggregates/mod.rs | 7 ++++ .../physical-plan/src/repartition/mod.rs | 7 ++++ 5 files changed, 87 insertions(+), 7 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index a73d084a081f..7fb33783c6b3 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -19,7 +19,8 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, - repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, + repartition_exec, repartition_selection_bitmap_exec, sort_exec, sort_expr_options, + sort_merge_join_exec, }; use arrow::compute::SortOptions; @@ -37,6 +38,8 @@ use datafusion_physical_plan::{displayable, ExecutionPlan}; use async_trait::async_trait; +use super::test_utils::final_partition_selection_bitmap_aggregate_exec; + async fn register_current_csv( ctx: &SessionContext, table_name: &str, @@ -647,3 +650,37 @@ async fn test_sort_merge_join_dist_missing() -> Result<()> { assert_sanity_check(&smj, false); Ok(()) } + +#[tokio::test] +async fn test_selection_bitmap_partitioning_agg() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let repartition = + repartition_selection_bitmap_exec(source, vec![col("c9", &schema).unwrap()]); + let plan = final_partition_selection_bitmap_aggregate_exec(repartition); + assert_plan( + plan.as_ref(), + vec![ + "AggregateExec: mode=FinalPartitioned(SelectionBitmap), gby=[], aggr=[]", + " RepartitionExec: partitioning=HashSelectionBitmap([c9@0], 1), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[0]" + ], + ); + assert_sanity_check(&plan, true); + + let schema = create_test_schema(); + let source = memory_exec(&schema); + let repartition = repartition_exec(source); + let plan = final_partition_selection_bitmap_aggregate_exec(repartition); + assert_plan( + plan.as_ref(), + vec![ + "AggregateExec: mode=FinalPartitioned(SelectionBitmap), gby=[], aggr=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[0]", + ], + ); + // final partition selection bitmap aggregate requires the input to be partitioned by selection bitmap. + assert_sanity_check(&plan, false); + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 0e36e8301420..41f8be02203d 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -41,7 +41,7 @@ use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr::{expressions, PhysicalExpr}; +use datafusion_physical_expr::{expressions, HashPartitionMode, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, PhysicalSortExpr, }; @@ -320,6 +320,16 @@ pub fn spr_repartition_exec(input: Arc) -> Arc, + keys: Vec>, +) -> Arc { + Arc::new( + RepartitionExec::try_new(input, Partitioning::HashSelectionBitmap(keys, 1)) + .unwrap(), + ) +} + pub fn aggregate_exec(input: Arc) -> Arc { let schema = input.schema(); Arc::new( @@ -335,6 +345,23 @@ pub fn aggregate_exec(input: Arc) -> Arc { ) } +pub fn final_partition_selection_bitmap_aggregate_exec( + input: Arc, +) -> Arc { + let schema = input.schema(); + Arc::new( + AggregateExec::try_new( + AggregateMode::FinalPartitioned(HashPartitionMode::SelectionBitmap), + PhysicalGroupBy::default(), + vec![], + vec![], + input, + schema, + ) + .unwrap(), + ) +} + pub fn coalesce_batches_exec(input: Arc) -> Arc { Arc::new(CoalesceBatchesExec::new(input, 128)) } diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index efb6e9254b52..ed1bd9fa806e 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -110,13 +110,15 @@ pub fn check_finiteness_requirements( pub fn check_filtered_batch_handle( plan: Arc, ) -> Result>> { - if plan.output_filtered_batches() { + if plan.requried_filtered_input() { for child in plan.children() { - if child.requried_filtered_input() { + if !child.output_filtered_batches() { + let plan_str = get_plan_string(&plan); + let child_str = get_plan_string(&child); return plan_err!( - "Plan {:?} prodouces filtered batches but its child {:?} won't consume them", - plan, - child + "Plan {:?} required filtered batches but its child {:?} won't produce them", + plan_str, + child_str ); } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 55af24c269d4..bea001765266 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1018,6 +1018,13 @@ impl ExecutionPlan for AggregateExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::LowerEqual } + + fn requried_filtered_input(&self) -> bool { + match self.mode() { + AggregateMode::FinalPartitioned(HashPartitionMode::SelectionBitmap) => true, + _ => false, + } + } } fn create_schema( diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 7d33bc1ce73b..e9026cfc2573 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -652,6 +652,13 @@ impl ExecutionPlan for RepartitionExec { Self::maintains_input_order_helper(self.input(), self.preserve_order) } + fn output_filtered_batches(&self) -> bool { + match self.partitioning() { + Partitioning::HashSelectionBitmap(_, _) => true, + _ => false, + } + } + fn execute( &self, partition: usize, From 4c83644f029ed7833957e397d49fad37d5b298bb Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Fri, 9 May 2025 00:40:54 +0800 Subject: [PATCH 35/37] fix clippy --- datafusion/physical-optimizer/src/sanity_checker.rs | 2 +- datafusion/physical-plan/src/aggregates/mod.rs | 5 +---- datafusion/physical-plan/src/repartition/mod.rs | 5 +---- 3 files changed, 3 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index ed1bd9fa806e..7f461bc34f7f 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -114,7 +114,7 @@ pub fn check_filtered_batch_handle( for child in plan.children() { if !child.output_filtered_batches() { let plan_str = get_plan_string(&plan); - let child_str = get_plan_string(&child); + let child_str = get_plan_string(child); return plan_err!( "Plan {:?} required filtered batches but its child {:?} won't produce them", plan_str, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bea001765266..50c36e17673b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1020,10 +1020,7 @@ impl ExecutionPlan for AggregateExec { } fn requried_filtered_input(&self) -> bool { - match self.mode() { - AggregateMode::FinalPartitioned(HashPartitionMode::SelectionBitmap) => true, - _ => false, - } + matches!(self.mode(), AggregateMode::FinalPartitioned(HashPartitionMode::SelectionBitmap)) } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index e9026cfc2573..ab0140b993f7 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -653,10 +653,7 @@ impl ExecutionPlan for RepartitionExec { } fn output_filtered_batches(&self) -> bool { - match self.partitioning() { - Partitioning::HashSelectionBitmap(_, _) => true, - _ => false, - } + matches!(self.partitioning(),Partitioning::HashSelectionBitmap(_, _)) } fn execute( From f76eba11122eaf0b38171dc29eab110012d114e3 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 10 May 2025 21:39:31 +0800 Subject: [PATCH 36/37] fix test and fmt --- datafusion/physical-optimizer/src/coalesce_batches.rs | 5 +---- datafusion/physical-plan/src/aggregates/mod.rs | 5 ++++- datafusion/physical-plan/src/repartition/mod.rs | 2 +- datafusion/sqllogictest/test_files/aggregate.slt | 9 ++++----- 4 files changed, 10 insertions(+), 11 deletions(-) diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 5cf2c877c61a..143f41186886 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -66,10 +66,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { || plan_any .downcast_ref::() .map(|repart_exec| { - !matches!( - repart_exec.partitioning().clone(), - Partitioning::RoundRobinBatch(_) - ) + !matches!(repart_exec.partitioning(), Partitioning::RoundRobinBatch(_) | Partitioning::HashSelectionBitmap(_, _)) }) .unwrap_or(false); if wrap_in_coalesce { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 50c36e17673b..2358847b1dfe 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1020,7 +1020,10 @@ impl ExecutionPlan for AggregateExec { } fn requried_filtered_input(&self) -> bool { - matches!(self.mode(), AggregateMode::FinalPartitioned(HashPartitionMode::SelectionBitmap)) + matches!( + self.mode(), + AggregateMode::FinalPartitioned(HashPartitionMode::SelectionBitmap) + ) } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index ab0140b993f7..a413a1b4d496 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -653,7 +653,7 @@ impl ExecutionPlan for RepartitionExec { } fn output_filtered_batches(&self) -> bool { - matches!(self.partitioning(),Partitioning::HashSelectionBitmap(_, _)) + matches!(self.partitioning(), Partitioning::HashSelectionBitmap(_, _)) } fn execute( diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 9eb03684d5a7..8ac19d165662 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1136,11 +1136,10 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] 05)--------AggregateExec: mode=FinalPartitioned(SelectionBitmap), gby=[alias1@0 as alias1], aggr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=HashSelectionBitmap([alias1@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=HashSelectionBitmap([alias1@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] # TODO: it should be 3 after fix: https://github.com/apache/datafusion/issues/15383 query I From 0a0055d3d6e24ed44e4d1ef1a183a21aefb86f13 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 10 May 2025 21:47:57 +0800 Subject: [PATCH 37/37] update config --- docs/source/user-guide/configs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 5f2233c8f9f3..fc30caabbbde 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -114,7 +114,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | | datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.prefer_hash_selection_vector_partitioning_agg | false | When set to true, the physical plan optimizer will prefer HashSelectionVectorPartitioning for HashAggregate over HashPartitioning. HashSelectionVectorPartitioning can work without data copying. | +| datafusion.optimizer.prefer_hash_selection_bitmap_partitioning_agg | false | When set to true, the physical plan optimizer will prefer HashSelectionBitmapPartitioning for HashAggregate over HashPartitioning. HashSelectionBitmapPartitioning can work without data copying. | | datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). |