From 80a4102452cfc975b88f29c9dbf25993ba8bb37d Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sun, 31 May 2026 15:44:16 +0800 Subject: [PATCH 1/8] Basic PartialHashAggregateStream --- datafusion/core/tests/sql/explain_analyze.rs | 4 +- .../src/aggregates/group_values/metrics.rs | 9 +- .../src/aggregates/group_values/mod.rs | 18 +- .../physical-plan/src/aggregates/mod.rs | 145 ++++- .../src/aggregates/raw_partial_hash.rs | 568 ++++++++++++++++++ .../sqllogictest/test_files/aggregate.slt | 9 + 6 files changed, 741 insertions(+), 12 deletions(-) create mode 100644 datafusion/physical-plan/src/aggregates/raw_partial_hash.rs diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 17e3dba14b90..080b666da792 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -776,9 +776,9 @@ 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, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)], stream=GroupedHashAggregateStream 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)] + AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)], stream=RawPartialHashAggregateStream FilterExec: c12@1 < 10 RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true diff --git a/datafusion/physical-plan/src/aggregates/group_values/metrics.rs b/datafusion/physical-plan/src/aggregates/group_values/metrics.rs index b6c32204e85f..10a08242e93e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/metrics.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/metrics.rs @@ -59,6 +59,7 @@ mod tests { use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::aggregate::AggregateExprBuilder; @@ -135,7 +136,13 @@ mod tests { schema, )?); - let task_ctx = Arc::new(TaskContext::default()); + // This test is for `GroupByMetrics`, which are maintained by + // `GroupedHashAggregateStream`. Use a finite memory pool so the partial + // aggregate does not take the raw-partial stream path. + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(10 * 1024 * 1024, 1.0) + .build_arc()?; + let task_ctx = Arc::new(TaskContext::default().with_runtime(runtime)); let _result = collect(Arc::clone(&aggregate_exec) as _, Arc::clone(&task_ctx)).await?; diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index ee253e5d7afd..3cbfa003ab2e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -134,6 +134,18 @@ pub trait GroupValues: Send { pub fn new_group_values( schema: SchemaRef, group_ordering: &GroupOrdering, +) -> Result> { + new_group_values_with_ordering(schema, !matches!(group_ordering, GroupOrdering::None)) +} + +/// Return a specialized unordered implementation of [`GroupValues`] for the given schema. +pub fn new_unordered_group_values(schema: SchemaRef) -> Result> { + new_group_values_with_ordering(schema, false) +} + +fn new_group_values_with_ordering( + schema: SchemaRef, + ordered: bool, ) -> Result> { if schema.fields.len() == 1 { let d = schema.fields[0].data_type(); @@ -201,10 +213,10 @@ pub fn new_group_values( } if multi_group_by::supported_schema(schema.as_ref()) { - if matches!(group_ordering, GroupOrdering::None) { - Ok(Box::new(GroupValuesColumn::::try_new(schema)?)) - } else { + if ordered { Ok(Box::new(GroupValuesColumn::::try_new(schema)?)) + } else { + Ok(Box::new(GroupValuesColumn::::try_new(schema)?)) } } else { Ok(Box::new(GroupValuesRows::try_new(schema)?)) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 541c27b5f2b8..05797d3a864f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -22,8 +22,8 @@ use std::sync::Arc; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; use crate::aggregates::{ - no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, - topk_stream::GroupedTopKAggregateStream, + no_grouping::AggregateStream, raw_partial_hash::RawPartialHashAggregateStream, + row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; use crate::execution_plan::{CardinalityEffect, EmissionType}; use crate::filter_pushdown::{ @@ -50,6 +50,7 @@ use datafusion_common::{ internal_err, not_impl_err, }; use datafusion_execution::TaskContext; +use datafusion_execution::memory_pool::MemoryLimit; use datafusion_expr::{Accumulator, Aggregate}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::equivalence::ProjectionMapping; @@ -71,6 +72,7 @@ use topk::heap::is_supported_heap_type; pub mod group_values; mod no_grouping; pub mod order; +mod raw_partial_hash; mod row_hash; mod topk; mod topk_stream; @@ -499,6 +501,7 @@ impl PartialEq for PhysicalGroupBy { #[expect(clippy::large_enum_variant)] enum StreamType { AggregateStream(AggregateStream), + RawPartialHash(RawPartialHashAggregateStream), GroupedHash(GroupedHashAggregateStream), GroupedPriorityQueue(GroupedTopKAggregateStream), } @@ -507,6 +510,7 @@ impl From for SendableRecordBatchStream { fn from(stream: StreamType) -> Self { match stream { StreamType::AggregateStream(stream) => Box::pin(stream), + StreamType::RawPartialHash(stream) => Box::pin(stream), StreamType::GroupedHash(stream) => Box::pin(stream), StreamType::GroupedPriorityQueue(stream) => Box::pin(stream), } @@ -964,12 +968,57 @@ impl AggregateExec { )); } + if self.should_use_raw_partial_hash_stream(context) { + return Ok(StreamType::RawPartialHash( + RawPartialHashAggregateStream::new(self, context, partition)?, + )); + } + // grouping by something else and we need to just materialize all results Ok(StreamType::GroupedHash(GroupedHashAggregateStream::new( self, context, partition, )?)) } + fn should_use_raw_partial_hash_stream(&self, context: &TaskContext) -> bool { + // Keep finite-memory execution on the existing stream for now because + // that path owns the mature spill / early-emission behavior. + if matches!(context.memory_pool().memory_limit(), MemoryLimit::Finite(_)) { + return false; + } + + self.can_use_raw_partial_hash_stream() + } + + fn can_use_raw_partial_hash_stream(&self) -> bool { + self.mode == AggregateMode::Partial + && self.limit_options.is_none() + && self.input_order_mode == InputOrderMode::Linear + && self.group_by.is_single() + } + + /// Returns the stream implied by plan shape for `EXPLAIN VERBOSE`. + /// + /// Execution also checks the runtime memory pool. Finite-memory execution + /// stays on `GroupedHashAggregateStream` to preserve its spill behavior. + fn stream_name_for_display(&self) -> &'static str { + if self.group_by.is_true_no_grouping() { + return "AggregateStream"; + } + + if self.limit_options.is_some() + && !self.is_unordered_unfiltered_group_by_distinct() + { + return "GroupedTopKAggregateStream"; + } + + if self.can_use_raw_partial_hash_stream() { + return "RawPartialHashAggregateStream"; + } + + "GroupedHashAggregateStream" + } + /// Finds the DataType and SortDirection for this Aggregate, if there is one pub fn get_minmax_desc(&self) -> Option<(FieldRef, bool)> { let agg_expr = self.aggr_expr.iter().exactly_one().ok()?; @@ -1448,6 +1497,10 @@ impl DisplayAs for AggregateExec { if self.input_order_mode != InputOrderMode::Linear { write!(f, ", ordering_mode={:?}", self.input_order_mode)?; } + + if matches!(t, DisplayFormatType::Verbose) { + write!(f, ", stream={}", self.stream_name_for_display())?; + } } DisplayFormatType::TreeRender => { let format_expr_with_alias = @@ -2180,6 +2233,32 @@ pub(crate) fn max_duplicate_ordinal(groups: &[Vec]) -> usize { /// The outer Vec appears to be for grouping sets /// The inner Vec contains the results per expression /// The inner-inner Array contains the results per row +/// +/// For example, for `GROUP BY GROUPING SETS ((a, b), (a))` with input: +/// +/// ```text +/// a b +/// 1 1 +/// 1 2 +/// 2 1 +/// ``` +/// +/// The output is: +/// +/// ```text +/// [ +/// [ +/// a: [1, 1, 2] +/// b: [1, 2, 1] +/// grouping_id: [0, 0, 0] +/// ], +/// [ +/// a: [1, 1, 2] +/// b: [NULL, NULL, NULL] +/// grouping_id: [1, 1, 1] +/// ] +/// ] +/// ``` pub fn evaluate_group_by( group_by: &PhysicalGroupBy, batch: &RecordBatch, @@ -2954,6 +3033,54 @@ mod tests { Ok(()) } + #[tokio::test] + async fn partial_grouped_aggregate_uses_raw_partial_stream() -> Result<()> { + let (schema, batches) = some_data(); + let input = TestMemoryExec::try_new_exec(&[batches], Arc::clone(&schema), None)?; + let group_by = + PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); + let udaf = Arc::new(AggregateUDF::from(InputTypeAssertingUdaf::new( + vec![DataType::Float64], + vec![DataType::Int32], + DataType::Int64, + ))); + let aggregates: Vec> = vec![Arc::new( + AggregateExprBuilder::new(udaf, vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("input_type_asserting(b)") + .build()?, + )]; + + let aggregate = AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggregates, + vec![None], + input, + Arc::clone(&schema), + )?; + let task_ctx = Arc::new( + TaskContext::default() + .with_session_config(SessionConfig::new().with_batch_size(2)), + ); + + let stream = aggregate.execute_typed(0, &task_ctx)?; + assert!(matches!(stream, StreamType::RawPartialHash(_))); + + let stream: SendableRecordBatchStream = stream.into(); + let batches = collect(stream).await?; + assert_eq!( + batches + .iter() + .map(RecordBatch::num_rows) + .collect::>(), + vec![2, 1] + ); + assert_eq!(batches.iter().map(RecordBatch::num_rows).sum::(), 3); + + Ok(()) + } + #[tokio::test] async fn test_drop_cancel_without_groups() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); @@ -3682,8 +3809,11 @@ mod tests { &ScalarValue::Float64(Some(0.1)), ); - let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let ctx = Arc::new(TaskContext::default().with_session_config(session_config)); + let stream: SendableRecordBatchStream = Box::pin( + GroupedHashAggregateStream::new(aggregate_exec.as_ref(), &ctx, 0)?, + ); + let output = collect(stream).await?; allow_duplicates! { assert_snapshot!(batches_to_string(&output), @r" @@ -3769,8 +3899,11 @@ mod tests { &ScalarValue::Float64(Some(0.1)), ); - let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let ctx = Arc::new(TaskContext::default().with_session_config(session_config)); + let stream: SendableRecordBatchStream = Box::pin( + GroupedHashAggregateStream::new(aggregate_exec.as_ref(), &ctx, 0)?, + ); + let output = collect(stream).await?; allow_duplicates! { assert_snapshot!(batches_to_string(&output), @r" diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs new file mode 100644 index 000000000000..65ac103976b5 --- /dev/null +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -0,0 +1,568 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Grouped hash aggregation for the first aggregation stage. +//! +//! This stream handles the basic grouped `raw rows -> partial state` path: +//! +//! ```text +//! input rows -> GROUP BY hash table -> accumulator state rows +//! ``` +//! +//! `AggregateExec` keeps finite-memory, ordered, limit, grouping-set, later-stage +//! (`partial state -> final values`, `partial state -> partial state`) and +//! single-stage aggregation on `GroupedHashAggregateStream` for now. + +use std::collections::HashMap; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use arrow::array::{ArrayRef, AsArray, BooleanArray, new_null_array}; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::Result; +use datafusion_execution::TaskContext; +use datafusion_execution::memory_pool::proxy::VecAllocExt; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_expr::{EmitTo, GroupsAccumulator}; +use futures::ready; +use futures::stream::{Stream, StreamExt}; + +use super::group_values::{GroupValues, new_unordered_group_values}; +use super::row_hash::create_group_accumulator; +use super::{ + AggregateExec, PhysicalGroupBy, aggregate_expressions, evaluate_group_by, + group_id_array, max_duplicate_ordinal, +}; +use crate::metrics::{ + BaselineMetrics, MetricBuilder, MetricCategory, RecordOutput, SpillMetrics, +}; +use crate::stream::EmptyRecordBatchStream; +use crate::{InputOrderMode, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; + +#[derive(Debug, Clone)] +enum ExecutionState { + ReadingInput, + ProducingOutput, + Done, +} + +struct RawPartialAccumulator { + /// Arguments to pass to this accumulator. + /// + /// Example: `CORR(x, y)` stores two expressions here, while `SUM(x)` stores one. + arguments: Vec>, + + /// Optional `FILTER` expression for this accumulator. + /// + /// Example: `SUM(x) FILTER (WHERE x > 10)` stores the `x > 10` predicate. + filter: Option>, + + /// Accumulator state for all groups for one aggregate expression. + accumulator: Box, +} + +struct EvaluatedRawPartialAccumulator { + arguments: Vec, + filter: Option, +} + +impl RawPartialAccumulator { + fn new( + arguments: Vec>, + filter: Option>, + accumulator: Box, + ) -> Self { + Self { + arguments, + filter, + accumulator, + } + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + let arguments = self + .arguments + .iter() + .map(|expr| { + expr.evaluate(batch) + .and_then(|value| value.into_array(batch.num_rows())) + }) + .collect::>()?; + + let filter = self + .filter + .as_ref() + .map(|filter| { + filter + .evaluate(batch) + .and_then(|value| value.into_array(batch.num_rows())) + }) + .transpose()?; + + Ok(EvaluatedRawPartialAccumulator { arguments, filter }) + } + + fn update_batch( + &mut self, + values: &EvaluatedRawPartialAccumulator, + group_indices: &[usize], + total_num_groups: usize, + ) -> Result<()> { + let filter = values.filter.as_ref().map(|filter| filter.as_boolean()); + self.accumulator.update_batch( + &values.arguments, + group_indices, + filter, + total_num_groups, + ) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + self.accumulator.state(emit_to) + } + + fn supports_convert_to_state(&self) -> bool { + self.accumulator.supports_convert_to_state() + } + + fn null_arguments(&self, input_schema: &SchemaRef) -> Result> { + self.arguments + .iter() + .map(|expr| { + let data_type = expr.data_type(input_schema)?; + Ok(new_null_array(&data_type, 1)) + }) + .collect() + } +} + +/// Hash table state for grouped raw-partial aggregation. +/// +/// This owns the coupled state for: +/// - evaluating group keys, +/// - interning each distinct group, +/// - mapping each input row to its group index, +/// - evaluating aggregate inputs, +/// - updating per-group accumulator state. +struct AggregateHashTable { + /// Raw input schema, used to evaluate expressions and synthesize empty + /// grouping-set rows. + input_schema: SchemaRef, + + /// Output schema: group columns followed by partial aggregate state columns. + output_schema: SchemaRef, + + /// Maximum rows per emitted output batch. + batch_size: usize, + + /// GROUP BY expressions evaluated for each input batch. + group_by: Arc, + + /// Interned group keys. Accumulator state is stored separately by group index. + group_values: Box, + + /// Group index for each row in the current input batch. + /// + /// Each value indexes into `group_values`, and the same index is used by every + /// accumulator to update that group's aggregate state. + batch_group_indices: Vec, + + /// One item per aggregate expression. + /// + /// Example: `COUNT(x), SUM(y)` creates two items. Each item owns the input + /// expressions, optional filter, and accumulator state for all groups. + accumulators: Vec, + + /// Full partial-state output built once after input is exhausted. + output_batch: Option, + + /// Offset of the next row to slice from `output_batch`. + output_batch_offset: usize, + + /// True once all output rows have been emitted. + output_finished: bool, +} + +impl AggregateHashTable { + fn new( + agg: &AggregateExec, + partition: usize, + output_schema: SchemaRef, + batch_size: usize, + ) -> Result { + let input_schema = agg.input().schema(); + let aggregate_arguments = aggregate_expressions( + &agg.aggr_expr, + &agg.mode, + agg.group_by.num_group_exprs(), + )?; + let accumulators: Vec<_> = agg + .aggr_expr + .iter() + .zip(aggregate_arguments) + .zip(agg.filter_expr.iter().cloned()) + .map(|((agg_expr, arguments), filter)| { + let accumulator = create_group_accumulator(agg_expr)?; + Ok(RawPartialAccumulator::new(arguments, filter, accumulator)) + }) + .collect::>()?; + + if accumulators + .iter() + .all(|acc| acc.supports_convert_to_state()) + { + let _skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) + .with_category(MetricCategory::Rows) + .counter("skipped_aggregation_rows", partition); + } + + let group_schema = agg.group_by.group_schema(&input_schema)?; + let group_values = new_unordered_group_values(group_schema)?; + + Ok(Self { + input_schema, + output_schema, + batch_size, + group_by: Arc::clone(&agg.group_by), + group_values, + batch_group_indices: Default::default(), + accumulators, + output_batch: None, + output_batch_offset: 0, + output_finished: false, + }) + } + + fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { + // outer vec: one per each grouping set + // inner vec: all group by exprs for the current grouping set + let grouping_set_args = evaluate_group_by(&self.group_by, batch)?; + // The evaluated args for each accumulator + let accumulator_args = self + .accumulators + .iter() + .map(|acc| acc.evaluate(batch)) + .collect::>>()?; + + for group_values in &grouping_set_args { + self.group_values + .intern(group_values, &mut self.batch_group_indices)?; + let group_indices = &self.batch_group_indices; + let total_num_groups = self.group_values.len(); + + for (acc, values) in self.accumulators.iter_mut().zip(accumulator_args.iter()) + { + acc.update_batch(values, group_indices, total_num_groups)?; + } + } + + Ok(()) + } + + fn next_output_batch(&mut self) -> Result> { + if self.output_finished { + return Ok(None); + } + + if self.output_batch.is_none() { + self.output_batch = self.build_output_batch()?; + self.output_batch_offset = 0; + } + + let Some(batch) = self.output_batch.as_ref() else { + self.output_finished = true; + return Ok(None); + }; + + debug_assert!(self.batch_size > 0); + let output_len = self + .batch_size + .max(1) + .min(batch.num_rows() - self.output_batch_offset); + let output = batch.slice(self.output_batch_offset, output_len); + self.output_batch_offset += output_len; + + if self.output_batch_offset == batch.num_rows() { + self.output_batch = None; + self.output_batch_offset = 0; + self.output_finished = true; + } + + debug_assert!(output.num_rows() > 0); + debug_assert!(output.num_rows() <= self.batch_size.max(1)); + Ok(Some(output)) + } + + fn build_output_batch(&mut self) -> Result> { + self.init_empty_grouping_sets()?; + + if self.group_values.is_empty() { + return Ok(None); + } + + let mut output = self.group_values.emit(EmitTo::All)?; + + for acc in self.accumulators.iter_mut() { + output.extend(acc.state(EmitTo::All)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + Ok(Some(batch)) + } + + fn memory_size(&self) -> usize { + let acc = self + .accumulators + .iter() + .map(|acc| acc.accumulator.size()) + .sum::(); + let output = self + .output_batch + .as_ref() + .map(RecordBatch::get_array_memory_size) + .unwrap_or_default(); + + acc + self.group_values.size() + + self.batch_group_indices.allocated_size() + + output + } + + fn clear(&mut self) { + self.group_values.clear_shrink(0); + self.batch_group_indices.clear(); + self.batch_group_indices.shrink_to(0); + self.output_batch = None; + self.output_batch_offset = 0; + self.output_finished = false; + } + + fn init_empty_grouping_sets(&mut self) -> Result<()> { + if !self.group_by.has_grouping_set() || !self.group_values.is_empty() { + return Ok(()); + } + + let max_ordinal = max_duplicate_ordinal(self.group_by.groups()); + let mut ordinals: HashMap<&[bool], usize> = HashMap::new(); + let group_schema = self.group_by.group_schema(&self.input_schema)?; + let n_expr = self.group_by.expr().len(); + let mut any_interned = false; + + for group in self.group_by.groups() { + let ordinal = { + let entry = ordinals.entry(group.as_slice()).or_insert(0); + let ordinal = *entry; + *entry += 1; + ordinal + }; + + if !group.iter().all(|&is_null| is_null) { + continue; + } + + let mut cols: Vec = group_schema + .fields() + .iter() + .take(n_expr) + .map(|field| new_null_array(field.data_type(), 1)) + .collect(); + cols.push(group_id_array(group, ordinal, max_ordinal, 1)?); + + self.group_values + .intern(&cols, &mut self.batch_group_indices)?; + any_interned = true; + } + + if any_interned { + let total_groups = self.group_values.len(); + let false_filter = BooleanArray::from(vec![false]); + for acc in self.accumulators.iter_mut() { + let null_args = acc.null_arguments(&self.input_schema)?; + let values = EvaluatedRawPartialAccumulator { + arguments: null_args, + filter: Some(Arc::new(false_filter.clone())), + }; + acc.update_batch(&values, &[0], total_groups)?; + } + } + + Ok(()) + } +} + +/// Hash aggregate stream for grouped `AggregateMode::Partial`. +/// +/// Input: raw rows +/// Output: partial state (e.g. for avg(x), it's sum(x), count(x)) +pub(crate) struct RawPartialHashAggregateStream { + // ======================================================================== + // PROPERTIES: + // Initialized once for this input partition. + // ======================================================================== + /// Output schema: group columns followed by partial aggregate state columns. + schema: SchemaRef, + + /// Input batches containing raw rows, not partial aggregate state. + input: SendableRecordBatchStream, + + // ======================================================================== + // STATE FLAGS: + // Control whether the stream is reading input, emitting state, or done. + // ======================================================================== + exec_state: ExecutionState, + + // ======================================================================== + // STATE BUFFERS: + // + // Hold intermediate groups and aggregate state while reading input. + // Example: `SELECT z, COUNT(x), SUM(y) FROM t GROUP BY z` stores each distinct + // `z` in `group_values` and keeps one partial-state accumulator for `COUNT(x)` + // and one for `SUM(y)`. + // ======================================================================== + /// Hash table and accumulator state for all groups seen so far. + hash_table: AggregateHashTable, + + // ======================================================================== + // EXECUTION RESOURCES: + // Metrics and memory accounting for this stream. + // ======================================================================== + /// Execution metrics shared with the aggregate plan node. + baseline_metrics: BaselineMetrics, + + /// Memory reservation for group keys and accumulators. + reservation: MemoryReservation, +} + +impl RawPartialHashAggregateStream { + pub fn new( + agg: &AggregateExec, + context: &Arc, + partition: usize, + ) -> Result { + debug_assert_eq!(agg.mode, super::AggregateMode::Partial); + debug_assert_eq!(agg.input_order_mode, InputOrderMode::Linear); + + let schema = Arc::clone(&agg.schema); + let input = agg.input.execute(partition, Arc::clone(context))?; + let batch_size = context.session_config().batch_size(); + let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); + + // Preserve the existing aggregate metric surface for this plan node. + let _spill_metrics = SpillMetrics::new(&agg.metrics, partition); + + let hash_table = + AggregateHashTable::new(agg, partition, Arc::clone(&schema), batch_size)?; + + let reservation = + MemoryConsumer::new(format!("RawPartialHashAggregateStream[{partition}]")) + .register(context.memory_pool()); + + Ok(Self { + schema, + input, + exec_state: ExecutionState::ReadingInput, + hash_table, + baseline_metrics, + reservation, + }) + } +} + +impl Stream for RawPartialHashAggregateStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + + loop { + match &self.exec_state { + ExecutionState::ReadingInput => { + match ready!(self.input.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + let timer = elapsed_compute.timer(); + let result = self.hash_table.aggregate_batch(&batch); + timer.done(); + + if let Err(e) = result { + return Poll::Ready(Some(Err(e))); + } + + // TODO: impl memory-limited aggr, when OOM directly send + // partial state to final aggregate stage + if let Err(e) = + self.reservation.try_resize(self.hash_table.memory_size()) + { + return Poll::Ready(Some(Err(e))); + } + } + Some(Err(e)) => { + return Poll::Ready(Some(Err(e))); + } + None => { + let input_schema = self.input.schema(); + self.input = + Box::pin(EmptyRecordBatchStream::new(input_schema)); + + self.exec_state = ExecutionState::ProducingOutput; + } + } + } + + ExecutionState::ProducingOutput => { + let timer = elapsed_compute.timer(); + let result = self.hash_table.next_output_batch(); + timer.done(); + + match result { + Ok(Some(batch)) => { + let _ = self + .reservation + .try_resize(self.hash_table.memory_size()); + debug_assert!(batch.num_rows() > 0); + return Poll::Ready(Some(Ok( + batch.record_output(&self.baseline_metrics) + ))); + } + Ok(None) => { + let _ = self + .reservation + .try_resize(self.hash_table.memory_size()); + self.exec_state = ExecutionState::Done; + } + Err(e) => return Poll::Ready(Some(Err(e))), + } + } + + ExecutionState::Done => { + self.hash_table.clear(); + let _ = self.reservation.try_resize(self.hash_table.memory_size()); + return Poll::Ready(None); + } + } + } + } +} + +impl RecordBatchStream for RawPartialHashAggregateStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 25b69d16dd03..a4316f2d0645 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -69,6 +69,15 @@ statement ok CREATE TABLE test (c1 BIGINT,c2 BIGINT) as values (0,null), (1,1), (null,1), (3,2), (3,2) +# EXPLAIN VERBOSE shows the concrete aggregate stream implementation. +query TT +EXPLAIN VERBOSE +SELECT value % 2 AS g, COUNT(*) +FROM generate_series(1, 3) +GROUP BY g; +---- +AggregateExec: mode=Partialstream=RawPartialHashAggregateStream + statement ok CREATE TABLE group_median_table_non_nullable ( col_group STRING NOT NULL, From b42ca5351d17d6aaf80ec6edf6405067c42e2474 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sun, 31 May 2026 17:43:49 +0800 Subject: [PATCH 2/8] basic PartialFinalAggregateStream --- datafusion/core/tests/dataframe/mod.rs | 156 +++--- .../enforce_distribution.rs | 128 ++--- .../physical_optimizer/filter_pushdown.rs | 8 +- .../partition_statistics.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 83 +++- .../src/aggregates/raw_partial_hash.rs | 446 +++++++++++++++--- .../sqllogictest/test_files/aggregate.slt | 36 +- .../test_files/aggregate_repartition.slt | 8 +- .../test_files/aggregates_simplify.slt | 8 +- .../test_files/aggregates_topk.slt | 16 +- .../sqllogictest/test_files/async_udf.slt | 4 +- .../sqllogictest/test_files/clickbench.slt | 144 +++--- .../test_files/count_star_rule.slt | 8 +- .../sqllogictest/test_files/distinct_on.slt | 4 +- .../sqllogictest/test_files/group_by.slt | 56 +-- .../test_files/grouping_set_repartition.slt | 14 +- .../test_files/information_schema.slt | 2 +- .../sqllogictest/test_files/join.slt.part | 4 +- datafusion/sqllogictest/test_files/joins.slt | 12 +- .../sqllogictest/test_files/lateral_join.slt | 4 +- datafusion/sqllogictest/test_files/limit.slt | 12 +- datafusion/sqllogictest/test_files/order.slt | 20 +- .../test_files/preserve_file_partitioning.slt | 16 +- .../test_files/projection_pushdown.slt | 4 +- .../test_files/push_down_filter_parquet.slt | 4 +- .../push_down_filter_regression.slt | 6 +- .../sqllogictest/test_files/qualify.slt | 8 +- .../sqllogictest/test_files/repartition.slt | 8 +- .../repartition_subset_satisfaction.slt | 8 +- datafusion/sqllogictest/test_files/select.slt | 4 +- .../sqllogictest/test_files/subquery.slt | 16 +- .../sqllogictest/test_files/subquery_sort.slt | 4 +- .../test_files/tpch/plans/q1.slt.part | 4 +- .../test_files/tpch/plans/q10.slt.part | 4 +- .../test_files/tpch/plans/q11.slt.part | 4 +- .../test_files/tpch/plans/q12.slt.part | 4 +- .../test_files/tpch/plans/q13.slt.part | 4 +- .../test_files/tpch/plans/q15.slt.part | 8 +- .../test_files/tpch/plans/q16.slt.part | 8 +- .../test_files/tpch/plans/q17.slt.part | 4 +- .../test_files/tpch/plans/q18.slt.part | 4 +- .../test_files/tpch/plans/q2.slt.part | 4 +- .../test_files/tpch/plans/q20.slt.part | 4 +- .../test_files/tpch/plans/q21.slt.part | 4 +- .../test_files/tpch/plans/q22.slt.part | 4 +- .../test_files/tpch/plans/q4.slt.part | 4 +- .../test_files/tpch/plans/q5.slt.part | 4 +- .../test_files/tpch/plans/q7.slt.part | 4 +- .../test_files/tpch/plans/q8.slt.part | 4 +- .../test_files/tpch/plans/q9.slt.part | 4 +- datafusion/sqllogictest/test_files/union.slt | 40 +- datafusion/sqllogictest/test_files/window.slt | 6 +- 53 files changed, 884 insertions(+), 503 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 0ced83f7b95f..61bc0e756039 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -3001,42 +3001,42 @@ async fn test_count_wildcard_on_sort() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) ASC NULLS LAST | - | | Projection: t1.b, count(Int64(1)) AS count(*) | - | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] | - | | TableScan: t1 projection=[b] | - | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | - | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | - | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*)] | - | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------+ + +---------------+----------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+----------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) ASC NULLS LAST | + | | Projection: t1.b, count(Int64(1)) AS count(*) | + | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] | + | | TableScan: t1 projection=[b] | + | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | + | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | + | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*)] | + | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+----------------------------------------------------------------------------------------------------------------------------+ " ); assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+---------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+---------------------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) AS 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(1) as count(*)] | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(1) as count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+---------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) AS 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(1) as count(*)], stream=PartialFinalHashAggregateStream | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(1) as count(*)], stream=RawPartialHashAggregateStream | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------------+ " ); Ok(()) @@ -3433,30 +3433,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+--------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+--------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+--------------------------------------------------------------------------------------------------------------------------+ + +---------------+--------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+--------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+--------------------------------------------------------------------------------------------------------------------------------+ " ); @@ -3488,30 +3488,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+--------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+--------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(1) as count(*)] | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(1) as count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+--------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(1) as count(*)], stream=PartialFinalHashAggregateStream | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(1) as count(*)], stream=RawPartialHashAggregateStream | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 426e1fa745e5..539a0cbbcb69 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1047,14 +1047,14 @@ fn join_after_agg_alias() -> Result<()> { plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)] - AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream 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, gby=[a2@0 as a2], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " @@ -1105,14 +1105,14 @@ fn hash_join_key_ordering() -> Result<()> { @r" 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, gby=[b1@0 as b1, a1@1 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream 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, gby=[b@0 as b, a@1 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " @@ -1829,16 +1829,16 @@ 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, gby=[b1@0 as b1, a1@1 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream 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, gby=[b@0 as b, a@1 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -1850,16 +1850,16 @@ 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, gby=[b1@0 as b1, a1@1 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream 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, gby=[b@0 as b, a@1 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -1937,17 +1937,17 @@ fn union_to_interleave() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] - AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[], stream=RawPartialHashAggregateStream InterleaveExec - AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream 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, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -1984,18 +1984,18 @@ fn union_not_to_interleave() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20 - AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[] + AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[], stream=RawPartialHashAggregateStream UnionExec - AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream 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, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -2014,9 +2014,9 @@ fn added_repartition_to_single_partition() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -2035,9 +2035,9 @@ fn repartition_deepest_node() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2139,9 +2139,9 @@ fn repartition_ignores_limit() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -2589,9 +2589,9 @@ fn parallelization_single_partition() -> Result<()> { test_config.to_plan(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet "); let plan_parquet_sort = test_config.to_plan(plan_parquet, &SORT_DISTRIB_DISTRIB); @@ -2601,9 +2601,9 @@ fn parallelization_single_partition() -> Result<()> { let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); let plan_csv_sort = test_config.to_plan(plan_csv, &SORT_DISTRIB_DISTRIB); @@ -2701,9 +2701,9 @@ fn parallelization_compressed_csv() -> Result<()> { // Compressed files cannot be partitioned assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); @@ -2711,9 +2711,9 @@ fn parallelization_compressed_csv() -> Result<()> { // Uncompressed files can be partitioned assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); } @@ -2740,9 +2740,9 @@ fn parallelization_two_partitions() -> Result<()> { test_config.to_plan(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet "); // Plan already has two partitions @@ -2752,9 +2752,9 @@ fn parallelization_two_partitions() -> Result<()> { // Test: with csv let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); // Plan already has two partitions @@ -2780,9 +2780,9 @@ fn parallelization_two_partitions_into_four() -> Result<()> { // Multiple source files split across partitions assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet "); // Multiple source files split across partitions @@ -2793,9 +2793,9 @@ fn parallelization_two_partitions_into_four() -> Result<()> { let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); // Multiple source files split across partitions @@ -2920,9 +2920,9 @@ fn parallelization_ignores_limit() -> Result<()> { test_config.to_plan(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -2942,9 +2942,9 @@ fn parallelization_ignores_limit() -> Result<()> { let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -3363,9 +3363,9 @@ fn preserve_ordering_for_streaming_sorted_aggregate() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted, stream=GroupedHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted, stream=GroupedHashAggregateStream DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet "); @@ -3397,9 +3397,9 @@ fn preserve_ordering_for_streaming_partially_sorted_aggregate() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]) + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]), stream=GroupedHashAggregateStream RepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC - AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]) + AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]), stream=GroupedHashAggregateStream DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet "); @@ -3633,8 +3633,8 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet "); let plan_sort = test_config.to_plan(physical_plan, &SORT_DISTRIB_DISTRIB); @@ -3663,11 +3663,11 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet "); diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index b420326596d0..92c93547c8d1 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -1848,7 +1848,7 @@ fn test_pushdown_filter_on_non_first_grouping_column() { OptimizationTest: input: - FilterExec: b@1 = bar - - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], stream=PartialFinalHashAggregateStream - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: @@ -2114,7 +2114,7 @@ fn test_pushdown_through_aggregate_with_reordered_input_columns() { OptimizationTest: input: - FilterExec: b@1 = bar - - AggregateExec: mode=Final, gby=[a@1 as a, b@2 as b], aggr=[cnt] + - AggregateExec: mode=Final, gby=[a@1 as a, b@2 as b], aggr=[cnt], stream=PartialFinalHashAggregateStream - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: @@ -2291,12 +2291,12 @@ fn test_pushdown_with_computed_grouping_key() { @r" OptimizationTest: input: - - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt] + - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt], stream=PartialFinalHashAggregateStream - FilterExec: c@2 > 5 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt] + - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt], stream=PartialFinalHashAggregateStream - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=c@2 > 5 " ); diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index d06e506abfeb..0ee2ed8ba215 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -826,7 +826,7 @@ mod test { let plan_string = get_plan_string(&aggregate_exec_partial).swap_remove(0); assert_snapshot!( plan_string, - @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)], stream=RawPartialHashAggregateStream" ); let p0_statistics = aggregate_exec_partial.partition_statistics(Some(0))?; @@ -922,7 +922,7 @@ mod test { let agg_plan = get_plan_string(&agg_partial).remove(0); assert_snapshot!( agg_plan, - @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)], stream=RawPartialHashAggregateStream" ); let empty_stat = Statistics { diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 080b666da792..f13a1a7c0de8 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -69,7 +69,8 @@ async fn explain_analyze_baseline_metrics() { assert_metrics!( &formatted, "AggregateExec: mode=Partial, gby=[c1@0 as c1]", - "reduction_factor=5.05% (5/99)" + "stream=RawPartialHashAggregateStream", + "skipped_aggregation_rows=0" ); { @@ -83,6 +84,7 @@ async fn explain_analyze_baseline_metrics() { assert_metrics!( &formatted, "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1]", + "stream=PartialFinalHashAggregateStream", "metrics=[output_rows=5, elapsed_compute=", "output_bytes=", expected_batch_count_after_repartition @@ -776,7 +778,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)], stream=GroupedHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)], stream=PartialFinalHashAggregateStream 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)], stream=RawPartialHashAggregateStream FilterExec: c12@1 < 10 diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 05797d3a864f..e3da94459db1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -22,8 +22,10 @@ use std::sync::Arc; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; use crate::aggregates::{ - no_grouping::AggregateStream, raw_partial_hash::RawPartialHashAggregateStream, - row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, + no_grouping::AggregateStream, + raw_partial_hash::{PartialFinalHashAggregateStream, RawPartialHashAggregateStream}, + row_hash::GroupedHashAggregateStream, + topk_stream::GroupedTopKAggregateStream, }; use crate::execution_plan::{CardinalityEffect, EmissionType}; use crate::filter_pushdown::{ @@ -502,6 +504,7 @@ impl PartialEq for PhysicalGroupBy { enum StreamType { AggregateStream(AggregateStream), RawPartialHash(RawPartialHashAggregateStream), + PartialFinalHash(PartialFinalHashAggregateStream), GroupedHash(GroupedHashAggregateStream), GroupedPriorityQueue(GroupedTopKAggregateStream), } @@ -511,6 +514,7 @@ impl From for SendableRecordBatchStream { match stream { StreamType::AggregateStream(stream) => Box::pin(stream), StreamType::RawPartialHash(stream) => Box::pin(stream), + StreamType::PartialFinalHash(stream) => Box::pin(stream), StreamType::GroupedHash(stream) => Box::pin(stream), StreamType::GroupedPriorityQueue(stream) => Box::pin(stream), } @@ -974,6 +978,12 @@ impl AggregateExec { )); } + if self.should_use_partial_final_hash_stream(context) { + return Ok(StreamType::PartialFinalHash( + PartialFinalHashAggregateStream::new(self, context, partition)?, + )); + } + // grouping by something else and we need to just materialize all results Ok(StreamType::GroupedHash(GroupedHashAggregateStream::new( self, context, partition, @@ -994,10 +1004,31 @@ impl AggregateExec { self.mode == AggregateMode::Partial && self.limit_options.is_none() && self.input_order_mode == InputOrderMode::Linear + && !self.group_by.is_true_no_grouping() && self.group_by.is_single() } - /// Returns the stream implied by plan shape for `EXPLAIN VERBOSE`. + fn should_use_partial_final_hash_stream(&self, context: &TaskContext) -> bool { + // Keep finite-memory execution on the existing stream for now because + // that path owns the mature spill behavior. + if matches!(context.memory_pool().memory_limit(), MemoryLimit::Finite(_)) { + return false; + } + + self.can_use_partial_final_hash_stream() + } + + fn can_use_partial_final_hash_stream(&self) -> bool { + matches!( + self.mode, + AggregateMode::Final | AggregateMode::FinalPartitioned + ) && self.limit_options.is_none() + && self.input_order_mode == InputOrderMode::Linear + && !self.group_by.is_true_no_grouping() + && self.group_by.is_single() + } + + /// Returns the stream implied by plan shape for `EXPLAIN`. /// /// Execution also checks the runtime memory pool. Finite-memory execution /// stays on `GroupedHashAggregateStream` to preserve its spill behavior. @@ -1016,9 +1047,19 @@ impl AggregateExec { return "RawPartialHashAggregateStream"; } + if self.can_use_partial_final_hash_stream() { + return "PartialFinalHashAggregateStream"; + } + "GroupedHashAggregateStream" } + fn should_display_stream_name(&self, t: DisplayFormatType) -> bool { + matches!(t, DisplayFormatType::Verbose) + || self.can_use_raw_partial_hash_stream() + || self.can_use_partial_final_hash_stream() + } + /// Finds the DataType and SortDirection for this Aggregate, if there is one pub fn get_minmax_desc(&self) -> Option<(FieldRef, bool)> { let agg_expr = self.aggr_expr.iter().exactly_one().ok()?; @@ -1498,7 +1539,7 @@ impl DisplayAs for AggregateExec { write!(f, ", ordering_mode={:?}", self.input_order_mode)?; } - if matches!(t, DisplayFormatType::Verbose) { + if self.should_display_stream_name(t) { write!(f, ", stream={}", self.stream_name_for_display())?; } } @@ -3051,20 +3092,20 @@ mod tests { .build()?, )]; - let aggregate = AggregateExec::try_new( + let partial_aggregate = Arc::new(AggregateExec::try_new( AggregateMode::Partial, - group_by, - aggregates, + group_by.clone(), + aggregates.clone(), vec![None], input, Arc::clone(&schema), - )?; + )?); let task_ctx = Arc::new( TaskContext::default() .with_session_config(SessionConfig::new().with_batch_size(2)), ); - let stream = aggregate.execute_typed(0, &task_ctx)?; + let stream = partial_aggregate.execute_typed(0, &task_ctx)?; assert!(matches!(stream, StreamType::RawPartialHash(_))); let stream: SendableRecordBatchStream = stream.into(); @@ -3078,6 +3119,30 @@ mod tests { ); assert_eq!(batches.iter().map(RecordBatch::num_rows).sum::(), 3); + let merge = Arc::new(CoalescePartitionsExec::new(partial_aggregate)); + let final_aggregate = AggregateExec::try_new( + AggregateMode::Final, + group_by.as_final(), + aggregates, + vec![None], + merge, + Arc::clone(&schema), + )?; + + let stream = final_aggregate.execute_typed(0, &task_ctx)?; + assert!(matches!(stream, StreamType::PartialFinalHash(_))); + + let stream: SendableRecordBatchStream = stream.into(); + let batches = collect(stream).await?; + assert_eq!( + batches + .iter() + .map(RecordBatch::num_rows) + .collect::>(), + vec![2, 1] + ); + assert_eq!(batches.iter().map(RecordBatch::num_rows).sum::(), 3); + Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs index 65ac103976b5..52f8b463bff4 100644 --- a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -15,19 +15,21 @@ // specific language governing permissions and limitations // under the License. -//! Grouped hash aggregation for the first aggregation stage. +//! Grouped hash aggregation for simple multi-stage aggregation paths. //! -//! This stream handles the basic grouped `raw rows -> partial state` path: +//! This module handles the basic grouped two-stage paths: //! //! ```text //! input rows -> GROUP BY hash table -> accumulator state rows +//! state rows -> GROUP BY hash table -> final aggregate rows //! ``` //! -//! `AggregateExec` keeps finite-memory, ordered, limit, grouping-set, later-stage -//! (`partial state -> final values`, `partial state -> partial state`) and -//! single-stage aggregation on `GroupedHashAggregateStream` for now. +//! `AggregateExec` keeps finite-memory, ordered, limit, grouping-set, +//! `partial state -> partial state`, and single-stage aggregation on +//! `GroupedHashAggregateStream` for now. use std::collections::HashMap; +use std::marker::PhantomData; use std::sync::Arc; use std::task::{Context, Poll}; @@ -42,7 +44,7 @@ use datafusion_expr::{EmitTo, GroupsAccumulator}; use futures::ready; use futures::stream::{Stream, StreamExt}; -use super::group_values::{GroupValues, new_unordered_group_values}; +use super::group_values::{GroupByMetrics, GroupValues, new_unordered_group_values}; use super::row_hash::create_group_accumulator; use super::{ AggregateExec, PhysicalGroupBy, aggregate_expressions, evaluate_group_by, @@ -61,7 +63,7 @@ enum ExecutionState { Done, } -struct RawPartialAccumulator { +struct HashAggregateAccumulator { /// Arguments to pass to this accumulator. /// /// Example: `CORR(x, y)` stores two expressions here, while `SUM(x)` stores one. @@ -76,12 +78,21 @@ struct RawPartialAccumulator { accumulator: Box, } -struct EvaluatedRawPartialAccumulator { +struct EvaluatedHashAggregateAccumulator { arguments: Vec, filter: Option, } -impl RawPartialAccumulator { +struct EvaluatedAggregateBatch { + /// One entry per grouping set; each entry contains all evaluated group key + /// arrays for the current input batch. + grouping_set_args: Vec>, + + /// Evaluated arguments and filters, one entry per aggregate expression. + accumulator_args: Vec, +} + +impl HashAggregateAccumulator { fn new( arguments: Vec>, filter: Option>, @@ -94,7 +105,7 @@ impl RawPartialAccumulator { } } - fn evaluate(&self, batch: &RecordBatch) -> Result { + fn evaluate(&self, batch: &RecordBatch) -> Result { let arguments = self .arguments .iter() @@ -114,12 +125,12 @@ impl RawPartialAccumulator { }) .transpose()?; - Ok(EvaluatedRawPartialAccumulator { arguments, filter }) + Ok(EvaluatedHashAggregateAccumulator { arguments, filter }) } fn update_batch( &mut self, - values: &EvaluatedRawPartialAccumulator, + values: &EvaluatedHashAggregateAccumulator, group_indices: &[usize], total_num_groups: usize, ) -> Result<()> { @@ -132,6 +143,25 @@ impl RawPartialAccumulator { ) } + fn merge_batch( + &mut self, + values: &EvaluatedHashAggregateAccumulator, + group_indices: &[usize], + total_num_groups: usize, + ) -> Result<()> { + debug_assert!(values.filter.is_none()); + self.accumulator.merge_batch( + &values.arguments, + group_indices, + None, + total_num_groups, + ) + } + + fn evaluate_final(&mut self, emit_to: EmitTo) -> Result { + self.accumulator.evaluate(emit_to) + } + fn state(&mut self, emit_to: EmitTo) -> Result> { self.accumulator.state(emit_to) } @@ -151,6 +181,12 @@ impl RawPartialAccumulator { } } +/// Hash table mode that consumes raw input rows and produces partial state. +struct RawPartial; + +/// Hash table mode that consumes partial state and produces final values. +struct PartialFinal; + /// Hash table state for grouped raw-partial aggregation. /// /// This owns the coupled state for: @@ -159,12 +195,15 @@ impl RawPartialAccumulator { /// - mapping each input row to its group index, /// - evaluating aggregate inputs, /// - updating per-group accumulator state. -struct AggregateHashTable { +struct AggregateHashTable { + /// Grouping and accumulator-specific timing metrics. + group_by_metrics: GroupByMetrics, + /// Raw input schema, used to evaluate expressions and synthesize empty /// grouping-set rows. input_schema: SchemaRef, - /// Output schema: group columns followed by partial aggregate state columns. + /// Output schema: group columns followed by aggregate state or final values. output_schema: SchemaRef, /// Maximum rows per emitted output batch. @@ -186,9 +225,9 @@ struct AggregateHashTable { /// /// Example: `COUNT(x), SUM(y)` creates two items. Each item owns the input /// expressions, optional filter, and accumulator state for all groups. - accumulators: Vec, + accumulators: Vec, - /// Full partial-state output built once after input is exhausted. + /// Full output built once after input is exhausted. output_batch: Option, /// Offset of the next row to slice from `output_batch`. @@ -196,14 +235,17 @@ struct AggregateHashTable { /// True once all output rows have been emitted. output_finished: bool, + + _mode: PhantomData, } -impl AggregateHashTable { - fn new( +impl AggregateHashTable { + fn new_with_filters( agg: &AggregateExec, partition: usize, output_schema: SchemaRef, batch_size: usize, + filters: Vec>>, ) -> Result { let input_schema = agg.input().schema(); let aggregate_arguments = aggregate_expressions( @@ -215,26 +257,22 @@ impl AggregateHashTable { .aggr_expr .iter() .zip(aggregate_arguments) - .zip(agg.filter_expr.iter().cloned()) + .zip(filters) .map(|((agg_expr, arguments), filter)| { let accumulator = create_group_accumulator(agg_expr)?; - Ok(RawPartialAccumulator::new(arguments, filter, accumulator)) + Ok(HashAggregateAccumulator::new( + arguments, + filter, + accumulator, + )) }) .collect::>()?; - if accumulators - .iter() - .all(|acc| acc.supports_convert_to_state()) - { - let _skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) - .with_category(MetricCategory::Rows) - .counter("skipped_aggregation_rows", partition); - } - let group_schema = agg.group_by.group_schema(&input_schema)?; let group_values = new_unordered_group_values(group_schema)?; Ok(Self { + group_by_metrics: GroupByMetrics::new(&agg.metrics, partition), input_schema, output_schema, batch_size, @@ -245,42 +283,42 @@ impl AggregateHashTable { output_batch: None, output_batch_offset: 0, output_finished: false, + _mode: PhantomData, }) } - fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { + fn evaluate_batch(&self, batch: &RecordBatch) -> Result { + let timer = self.group_by_metrics.time_calculating_group_ids.timer(); // outer vec: one per each grouping set // inner vec: all group by exprs for the current grouping set let grouping_set_args = evaluate_group_by(&self.group_by, batch)?; + drop(timer); + + let timer = self.group_by_metrics.aggregate_arguments_time.timer(); // The evaluated args for each accumulator let accumulator_args = self .accumulators .iter() .map(|acc| acc.evaluate(batch)) .collect::>>()?; + drop(timer); - for group_values in &grouping_set_args { - self.group_values - .intern(group_values, &mut self.batch_group_indices)?; - let group_indices = &self.batch_group_indices; - let total_num_groups = self.group_values.len(); - - for (acc, values) in self.accumulators.iter_mut().zip(accumulator_args.iter()) - { - acc.update_batch(values, group_indices, total_num_groups)?; - } - } - - Ok(()) + Ok(EvaluatedAggregateBatch { + grouping_set_args, + accumulator_args, + }) } - fn next_output_batch(&mut self) -> Result> { + fn next_output_batch_from( + &mut self, + build_output_batch: impl FnOnce(&mut Self) -> Result>, + ) -> Result> { if self.output_finished { return Ok(None); } if self.output_batch.is_none() { - self.output_batch = self.build_output_batch()?; + self.output_batch = build_output_batch(self)?; self.output_batch_offset = 0; } @@ -308,24 +346,6 @@ impl AggregateHashTable { Ok(Some(output)) } - fn build_output_batch(&mut self) -> Result> { - self.init_empty_grouping_sets()?; - - if self.group_values.is_empty() { - return Ok(None); - } - - let mut output = self.group_values.emit(EmitTo::All)?; - - for acc in self.accumulators.iter_mut() { - output.extend(acc.state(EmitTo::All)?); - } - - let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; - debug_assert!(batch.num_rows() > 0); - Ok(Some(batch)) - } - fn memory_size(&self) -> usize { let acc = self .accumulators @@ -351,6 +371,82 @@ impl AggregateHashTable { self.output_batch_offset = 0; self.output_finished = false; } +} + +impl AggregateHashTable { + fn new( + agg: &AggregateExec, + partition: usize, + output_schema: SchemaRef, + batch_size: usize, + ) -> Result { + let table = Self::new_with_filters( + agg, + partition, + output_schema, + batch_size, + agg.filter_expr.iter().cloned().collect(), + )?; + + if table + .accumulators + .iter() + .all(|acc| acc.supports_convert_to_state()) + { + let _skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) + .with_category(MetricCategory::Rows) + .counter("skipped_aggregation_rows", partition); + } + + Ok(table) + } + + fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { + let evaluated_batch = self.evaluate_batch(batch)?; + + let timer = self.group_by_metrics.aggregation_time.timer(); + for group_values in &evaluated_batch.grouping_set_args { + self.group_values + .intern(group_values, &mut self.batch_group_indices)?; + let group_indices = &self.batch_group_indices; + let total_num_groups = self.group_values.len(); + + for (acc, values) in self + .accumulators + .iter_mut() + .zip(evaluated_batch.accumulator_args.iter()) + { + acc.update_batch(values, group_indices, total_num_groups)?; + } + } + drop(timer); + + Ok(()) + } + + fn next_output_batch(&mut self) -> Result> { + self.next_output_batch_from(Self::build_output_batch) + } + + fn build_output_batch(&mut self) -> Result> { + self.init_empty_grouping_sets()?; + + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::All)?; + + for acc in self.accumulators.iter_mut() { + output.extend(acc.state(EmitTo::All)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + drop(timer); + Ok(Some(batch)) + } fn init_empty_grouping_sets(&mut self) -> Result<()> { if !self.group_by.has_grouping_set() || !self.group_values.is_empty() { @@ -393,7 +489,7 @@ impl AggregateHashTable { let false_filter = BooleanArray::from(vec![false]); for acc in self.accumulators.iter_mut() { let null_args = acc.null_arguments(&self.input_schema)?; - let values = EvaluatedRawPartialAccumulator { + let values = EvaluatedHashAggregateAccumulator { arguments: null_args, filter: Some(Arc::new(false_filter.clone())), }; @@ -405,8 +501,70 @@ impl AggregateHashTable { } } +impl AggregateHashTable { + fn new( + agg: &AggregateExec, + partition: usize, + output_schema: SchemaRef, + batch_size: usize, + ) -> Result { + Self::new_with_filters( + agg, + partition, + output_schema, + batch_size, + vec![None; agg.aggr_expr.len()], + ) + } + + fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { + let evaluated_batch = self.evaluate_batch(batch)?; + + let timer = self.group_by_metrics.aggregation_time.timer(); + for group_values in &evaluated_batch.grouping_set_args { + self.group_values + .intern(group_values, &mut self.batch_group_indices)?; + let group_indices = &self.batch_group_indices; + let total_num_groups = self.group_values.len(); + + for (acc, values) in self + .accumulators + .iter_mut() + .zip(evaluated_batch.accumulator_args.iter()) + { + acc.merge_batch(values, group_indices, total_num_groups)?; + } + } + drop(timer); + + Ok(()) + } + + fn next_output_batch(&mut self) -> Result> { + self.next_output_batch_from(Self::build_output_batch) + } + + fn build_output_batch(&mut self) -> Result> { + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::All)?; + + for acc in self.accumulators.iter_mut() { + output.push(acc.evaluate_final(EmitTo::All)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + drop(timer); + Ok(Some(batch)) + } +} + /// Hash aggregate stream for grouped `AggregateMode::Partial`. -/// +/// /// Input: raw rows /// Output: partial state (e.g. for avg(x), it's sum(x), count(x)) pub(crate) struct RawPartialHashAggregateStream { @@ -435,7 +593,7 @@ pub(crate) struct RawPartialHashAggregateStream { // and one for `SUM(y)`. // ======================================================================== /// Hash table and accumulator state for all groups seen so far. - hash_table: AggregateHashTable, + hash_table: AggregateHashTable, // ======================================================================== // EXECUTION RESOURCES: @@ -465,8 +623,12 @@ impl RawPartialHashAggregateStream { // Preserve the existing aggregate metric surface for this plan node. let _spill_metrics = SpillMetrics::new(&agg.metrics, partition); - let hash_table = - AggregateHashTable::new(agg, partition, Arc::clone(&schema), batch_size)?; + let hash_table = AggregateHashTable::::new( + agg, + partition, + Arc::clone(&schema), + batch_size, + )?; let reservation = MemoryConsumer::new(format!("RawPartialHashAggregateStream[{partition}]")) @@ -566,3 +728,151 @@ impl RecordBatchStream for RawPartialHashAggregateStream { Arc::clone(&self.schema) } } + +/// `AggregateMode::FinalPartitioned`. +/// +/// Input: partial state, such as `sum(x), count(x)` for `avg(x)`. +/// Output: final values, such as `avg(x)`. +pub(crate) struct PartialFinalHashAggregateStream { + /// Output schema: group columns followed by final aggregate value columns. + schema: SchemaRef, + + /// Input batches containing partial aggregate state rows. + input: SendableRecordBatchStream, + + /// Controls whether the stream is reading input, emitting output, or done. + exec_state: ExecutionState, + + /// Hash table and accumulator state for all groups seen so far. + hash_table: AggregateHashTable, + + /// Execution metrics shared with the aggregate plan node. + baseline_metrics: BaselineMetrics, + + /// Memory reservation for group keys and accumulators. + reservation: MemoryReservation, +} + +impl PartialFinalHashAggregateStream { + pub fn new( + agg: &AggregateExec, + context: &Arc, + partition: usize, + ) -> Result { + debug_assert!(matches!( + agg.mode, + super::AggregateMode::Final | super::AggregateMode::FinalPartitioned + )); + debug_assert_eq!(agg.input_order_mode, InputOrderMode::Linear); + + let schema = Arc::clone(&agg.schema); + let input = agg.input.execute(partition, Arc::clone(context))?; + let batch_size = context.session_config().batch_size(); + let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); + + // Preserve the existing aggregate metric surface for this plan node. + let _spill_metrics = SpillMetrics::new(&agg.metrics, partition); + + let hash_table = AggregateHashTable::::new( + agg, + partition, + Arc::clone(&schema), + batch_size, + )?; + + let reservation = + MemoryConsumer::new(format!("PartialFinalHashAggregateStream[{partition}]")) + .register(context.memory_pool()); + + Ok(Self { + schema, + input, + exec_state: ExecutionState::ReadingInput, + hash_table, + baseline_metrics, + reservation, + }) + } +} + +impl Stream for PartialFinalHashAggregateStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + + loop { + match &self.exec_state { + ExecutionState::ReadingInput => { + match ready!(self.input.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + let timer = elapsed_compute.timer(); + let result = self.hash_table.aggregate_batch(&batch); + timer.done(); + + if let Err(e) = result { + return Poll::Ready(Some(Err(e))); + } + + if let Err(e) = + self.reservation.try_resize(self.hash_table.memory_size()) + { + return Poll::Ready(Some(Err(e))); + } + } + Some(Err(e)) => { + return Poll::Ready(Some(Err(e))); + } + None => { + let input_schema = self.input.schema(); + self.input = + Box::pin(EmptyRecordBatchStream::new(input_schema)); + + self.exec_state = ExecutionState::ProducingOutput; + } + } + } + + ExecutionState::ProducingOutput => { + let timer = elapsed_compute.timer(); + let result = self.hash_table.next_output_batch(); + timer.done(); + + match result { + Ok(Some(batch)) => { + let _ = self + .reservation + .try_resize(self.hash_table.memory_size()); + debug_assert!(batch.num_rows() > 0); + return Poll::Ready(Some(Ok( + batch.record_output(&self.baseline_metrics) + ))); + } + Ok(None) => { + let _ = self + .reservation + .try_resize(self.hash_table.memory_size()); + self.exec_state = ExecutionState::Done; + } + Err(e) => return Poll::Ready(Some(Err(e))), + } + } + + ExecutionState::Done => { + self.hash_table.clear(); + let _ = self.reservation.try_resize(self.hash_table.memory_size()); + return Poll::Ready(None); + } + } + } + } +} + +impl RecordBatchStream for PartialFinalHashAggregateStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index a4316f2d0645..c9df9c64a459 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -69,13 +69,17 @@ statement ok CREATE TABLE test (c1 BIGINT,c2 BIGINT) as values (0,null), (1,1), (null,1), (3,2), (3,2) -# EXPLAIN VERBOSE shows the concrete aggregate stream implementation. +# Regular indent EXPLAIN shows the concrete aggregate stream implementation. +statement ok +set datafusion.explain.format = 'indent'; + query TT -EXPLAIN VERBOSE +EXPLAIN SELECT value % 2 AS g, COUNT(*) FROM generate_series(1, 3) GROUP BY g; ---- +AggregateExec: mode=FinalPartitionedstream=PartialFinalHashAggregateStream AggregateExec: mode=Partialstream=RawPartialHashAggregateStream statement ok @@ -1681,9 +1685,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, gby=[alias1@0 as alias1], aggr=[] +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[CAST(c@0 AS Float64) as alias1], aggr=[] +07)------------AggregateExec: mode=Partial, gby=[CAST(c@0 AS Float64) as alias1], aggr=[], stream=RawPartialHashAggregateStream 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok @@ -6539,9 +6543,9 @@ logical_plan physical_plan 01)CoalescePartitionsExec: fetch=5 02)--AggregateExec: mode=SinglePartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] +03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 -05)--------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] +05)--------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)], stream=RawPartialHashAggregateStream 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c1, c3], file_type=csv, has_header=true @@ -6621,9 +6625,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[4] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[c3@1 as c3] -07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=PartialFinalHashAggregateStream 08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=RawPartialHashAggregateStream 10)------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -6648,9 +6652,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(aggregate_test_100.c1)@2 as max(aggregate_test_100.c1), c2@0 as c2, c3@1 as c3] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[max(aggregate_test_100.c1)] +03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[max(aggregate_test_100.c1)], stream=PartialFinalHashAggregateStream 04)------CoalescePartitionsExec -05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)] +05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)], stream=RawPartialHashAggregateStream 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true @@ -6672,9 +6676,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] -07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=PartialFinalHashAggregateStream 08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=RawPartialHashAggregateStream 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -6723,9 +6727,9 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c3] physical_plan 01)GlobalLimitExec: skip=0, fetch=5 -02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] +02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], stream=PartialFinalHashAggregateStream 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] +04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], stream=RawPartialHashAggregateStream 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c3], file_type=csv, has_header=true @@ -7777,9 +7781,9 @@ logical_plan 04)------TableScan: having_test projection=[v1, v2] physical_plan 01)FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] -02)--AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +02)--AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +04)------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/aggregate_repartition.slt b/datafusion/sqllogictest/test_files/aggregate_repartition.slt index 1f1e72681167..b5ddbf02feaf 100644 --- a/datafusion/sqllogictest/test_files/aggregate_repartition.slt +++ b/datafusion/sqllogictest/test_files/aggregate_repartition.slt @@ -76,9 +76,9 @@ logical_plan 03)----TableScan: dim_csv projection=[env] physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] -02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] +02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=4 -04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] +04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.csv]]}, projection=[env], file_type=csv, has_header=true @@ -93,9 +93,9 @@ logical_plan 03)----TableScan: dim_parquet projection=[env] physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] -02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] +02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] +04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet # Verify the queries actually work and return the same results diff --git a/datafusion/sqllogictest/test_files/aggregates_simplify.slt b/datafusion/sqllogictest/test_files/aggregates_simplify.slt index c4055d17396c..7e07cd24f4f9 100644 --- a/datafusion/sqllogictest/test_files/aggregates_simplify.slt +++ b/datafusion/sqllogictest/test_files/aggregates_simplify.slt @@ -170,9 +170,9 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(alias1), sum(alias2)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(alias1), sum(alias2)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[sum(__common_expr_1) as alias2] +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[sum(__common_expr_1) as alias2], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as alias1], aggr=[sum(__common_expr_1) as alias2] +07)------------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as alias1], aggr=[sum(__common_expr_1) as alias2], stream=RawPartialHashAggregateStream 08)--------------ProjectionExec: expr=[column1@0 + 1 as __common_expr_1] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -273,9 +273,9 @@ physical_plan 01)SortPreservingMergeExec: [column2@0 DESC NULLS LAST] 02)--SortExec: expr=[column2@0 DESC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[column2@0 as column2, sum(sum_simplify_t.column1)@1 + count(sum_simplify_t.column1)@2 as sum(sum_simplify_t.column1 + Int64(1)), sum(sum_simplify_t.column1)@1 + 2 * count(sum_simplify_t.column1)@2 as sum(sum_simplify_t.column1 + Int64(2))] -04)------AggregateExec: mode=FinalPartitioned, gby=[column2@0 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[column2@0 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([column2@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[column2@1 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)] +06)----------AggregateExec: mode=Partial, gby=[column2@1 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: partitions=1, partition_sizes=[1] # Checks commutative forms of equivalent aggregate arguments are simplified consistently. diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 19ead8965ed0..947a02ed690c 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -45,9 +45,9 @@ 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, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TI @@ -122,9 +122,9 @@ 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, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -137,9 +137,9 @@ 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, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -152,9 +152,9 @@ 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, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TI diff --git a/datafusion/sqllogictest/test_files/async_udf.slt b/datafusion/sqllogictest/test_files/async_udf.slt index 678b2f7d8b8d..07f985a99ce4 100644 --- a/datafusion/sqllogictest/test_files/async_udf.slt +++ b/datafusion/sqllogictest/test_files/async_udf.slt @@ -56,9 +56,9 @@ logical_plan 04)------TableScan: data projection=[x] physical_plan 01)ProjectionExec: expr=[min(async_abs(data.x))@1 as min(async_abs(data.x))] -02)--AggregateExec: mode=FinalPartitioned, gby=[async_abs(data.x)@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))] +02)--AggregateExec: mode=FinalPartitioned, gby=[async_abs(data.x)@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([async_abs(data.x)@0], 4), input_partitions=4 -04)------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))] +04)------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))], stream=RawPartialHashAggregateStream 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[__async_fn_0@1 as __common_expr_1] 07)------------AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=async_abs(x@0))] diff --git a/datafusion/sqllogictest/test_files/clickbench.slt b/datafusion/sqllogictest/test_files/clickbench.slt index 91463c9c2bff..5155b5205f41 100644 --- a/datafusion/sqllogictest/test_files/clickbench.slt +++ b/datafusion/sqllogictest/test_files/clickbench.slt @@ -147,9 +147,9 @@ 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, gby=[alias1@0 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[] +07)------------AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet query I @@ -172,9 +172,9 @@ 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, gby=[alias1@0 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[] +07)------------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet query I @@ -215,9 +215,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@1 DESC] 02)--SortExec: expr=[count(*)@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(Int64(1))@1 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([AdvEngineID@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------FilterExec: AdvEngineID@0 != 0 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] @@ -241,12 +241,12 @@ physical_plan 01)SortPreservingMergeExec: [u@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[RegionID@0 as RegionID, count(alias1)@1 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([RegionID@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] -07)------------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 08)--------------RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[] +09)----------------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[], stream=RawPartialHashAggregateStream 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[RegionID, UserID], file_type=parquet query II rowsort @@ -271,9 +271,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[RegionID@0 as RegionID, sum(hits.AdvEngineID)@1 as sum(hits.AdvEngineID), count(Int64(1))@2 as c, avg(hits.ResolutionWidth)@3 as avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] -04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] +04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([RegionID@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] +06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[RegionID, UserID, ResolutionWidth, AdvEngineID], file_type=parquet query IIIRI rowsort @@ -300,12 +300,12 @@ physical_plan 01)SortPreservingMergeExec: [u@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[MobilePhoneModel@0 as MobilePhoneModel, count(alias1)@1 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([MobilePhoneModel@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] -07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 08)--------------RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] +09)----------------AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream 10)------------------FilterExec: MobilePhoneModel@1 != 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] @@ -330,12 +330,12 @@ physical_plan 01)SortPreservingMergeExec: [u@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, count(alias1)@2 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] -07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 08)--------------RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] +09)----------------AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream 10)------------------FilterExec: MobilePhoneModel@2 != 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] @@ -359,9 +359,9 @@ physical_plan 01)SortPreservingMergeExec: [c@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(Int64(1))@1 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------FilterExec: SearchPhrase@0 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -386,12 +386,12 @@ physical_plan 01)SortPreservingMergeExec: [u@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(alias1)@1 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] -07)------------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 08)--------------RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] +09)----------------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream 10)------------------FilterExec: SearchPhrase@1 != 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -415,9 +415,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------FilterExec: SearchPhrase@1 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -440,9 +440,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(*)@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[UserID@0 as UserID, count(Int64(1))@1 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([UserID@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet query II rowsort @@ -468,9 +468,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(*)@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet query ITI rowsort @@ -495,9 +495,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*)] 02)--CoalescePartitionsExec: fetch=10 -03)----AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] +03)----AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet query ITI rowsort @@ -523,9 +523,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@3 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(*)@3 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as m, SearchPhrase@2 as SearchPhrase, count(Int64(1))@3 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, UserID, SearchPhrase], file_type=parquet query IITI rowsort @@ -599,9 +599,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, min(hits.URL)@1 as min(hits.URL), count(Int64(1))@2 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------FilterExec: URL@0 LIKE %google% AND SearchPhrase@1 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL, SearchPhrase], file_type=parquet, predicate=URL@13 LIKE %google% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@4 != row_count@5 AND (SearchPhrase_min@2 != OR != SearchPhrase_max@3), required_guarantees=[SearchPhrase not in ()] @@ -625,9 +625,9 @@ physical_plan 01)SortPreservingMergeExec: [c@3 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@3 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, min(hits.URL)@1 as min(hits.URL), min(hits.Title)@2 as min(hits.Title), count(Int64(1))@3 as c, count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@3 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@3 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)], stream=RawPartialHashAggregateStream 07)------------FilterExec: Title@0 LIKE %Google% AND URL@2 NOT LIKE %.google.% AND SearchPhrase@3 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Title, UserID, URL, SearchPhrase], file_type=parquet, predicate=Title@2 LIKE %Google% AND URL@13 NOT LIKE %.google.% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@6 != row_count@7 AND (SearchPhrase_min@4 != OR != SearchPhrase_max@5), required_guarantees=[SearchPhrase not in ()] @@ -741,9 +741,9 @@ physical_plan 02)--SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CounterID@0 as CounterID, avg(length(hits.URL))@1 as l, count(Int64(1))@2 as c] 04)------FilterExec: count(Int64(1))@2 > 100000 -05)--------AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([CounterID@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------FilterExec: URL@1 != 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[CounterID, URL], file_type=parquet, predicate=URL@13 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] @@ -769,9 +769,9 @@ physical_plan 02)--SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as k, avg(length(hits.Referer))@1 as l, count(Int64(1))@2 as c, min(hits.Referer)@3 as min(hits.Referer)] 04)------FilterExec: count(Int64(1))@2 > 100000 -05)--------AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[regexp_replace(Referer@0, ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] +07)------------AggregateExec: mode=Partial, gby=[regexp_replace(Referer@0, ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)], stream=RawPartialHashAggregateStream 08)--------------FilterExec: Referer@0 != 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Referer], file_type=parquet, predicate=Referer@14 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] @@ -817,9 +817,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] +06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=RawPartialHashAggregateStream 07)------------FilterExec: SearchPhrase@4 != , projection=[ClientIP@0, IsRefresh@1, ResolutionWidth@2, SearchEngineID@3] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -844,9 +844,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[WatchID@0 as WatchID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] -04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] +04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] +06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=RawPartialHashAggregateStream 07)------------FilterExec: SearchPhrase@4 != , projection=[WatchID@0, ClientIP@1, IsRefresh@2, ResolutionWidth@3] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -869,9 +869,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[WatchID@0 as WatchID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] -04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] +04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] +06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth], file_type=parquet query IIIIR rowsort @@ -902,9 +902,9 @@ physical_plan 01)SortPreservingMergeExec: [c@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet query TI rowsort @@ -931,9 +931,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[1 as Int64(1), URL@0 as URL, count(Int64(1))@1 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet query ITI rowsort @@ -962,9 +962,9 @@ physical_plan 02)--SortExec: TopK(fetch=10), expr=[c@4 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[ClientIP@1 as ClientIP, __common_expr_1@0 - 1 as hits.ClientIP - Int64(1), __common_expr_1@0 - 2 as hits.ClientIP - Int64(2), __common_expr_1@0 - 3 as hits.ClientIP - Int64(3), count(Int64(1))@2 as c] 04)------ProjectionExec: expr=[CAST(ClientIP@0 AS Int64) as __common_expr_1, ClientIP@0 as ClientIP, count(Int64(1))@1 as count(Int64(1))] -05)--------AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([ClientIP@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[ClientIP], file_type=parquet query IIIII rowsort @@ -991,9 +991,9 @@ physical_plan 01)SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] -04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , projection=[URL@2] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND URL@13 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND URL_null_count@15 != row_count@3 AND (URL_min@13 != OR != URL_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] @@ -1018,9 +1018,9 @@ physical_plan 01)SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[Title@0 as Title, count(Int64(1))@1 as pageviews] -04)------AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))] +04)------AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([Title@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------FilterExec: CounterID@2 = 62 AND EventDate@1 >= 15887 AND EventDate@1 <= 15917 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , projection=[Title@0] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND Title@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND Title_null_count@15 != row_count@3 AND (Title_min@13 != OR != Title_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] @@ -1047,9 +1047,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@1 DESC], fetch=1010 03)----SortExec: TopK(fetch=1010), expr=[pageviews@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, projection=[URL@2] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND IsRefresh@15 = 0 AND IsLink@52 != 0 AND IsDownload@53 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND IsLink_null_count@12 != row_count@3 AND (IsLink_min@10 != 0 OR 0 != IsLink_max@11) AND IsDownload_null_count@15 != row_count@3 AND IsDownload_min@13 <= 0 AND 0 <= IsDownload_max@14, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] @@ -1076,9 +1076,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@5 DESC], fetch=1010 03)----SortExec: TopK(fetch=1010), expr=[pageviews@5 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as src, URL@4 as dst, count(Int64(1))@5 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@4 = 0, projection=[URL@2, Referer@3, TraficSourceID@5, SearchEngineID@6, AdvEngineID@7] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND IsRefresh@15 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8, required_guarantees=[CounterID in (62), IsRefresh in (0)] @@ -1105,9 +1105,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@2 DESC], fetch=110 03)----SortExec: TopK(fetch=110), expr=[pageviews@2 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[URLHash@0 as URLHash, EventDate@1 as EventDate, count(Int64(1))@2 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------ProjectionExec: expr=[URLHash@0 as URLHash, CAST(CAST(EventDate@1 AS Int32) AS Date32) as EventDate] 09)----------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[URLHash@5, EventDate@0] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -1135,9 +1135,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@2 DESC], fetch=10010 03)----SortExec: TopK(fetch=10010), expr=[pageviews@2 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight, count(Int64(1))@2 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, projection=[WindowClientWidth@3, WindowClientHeight@4] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0 AND URLHash@103 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11 AND URLHash_null_count@15 != row_count@3 AND URLHash_min@13 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@14, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] @@ -1164,9 +1164,9 @@ physical_plan 02)--SortPreservingMergeExec: [date_trunc(minute, m@0) ASC NULLS LAST], fetch=1010 03)----SortExec: TopK(fetch=1010), expr=[date_trunc(minute, m@0) ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as m, count(Int64(1))@1 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[date_trunc(minute, to_timestamp_seconds(EventTime@0)) as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] +07)------------AggregateExec: mode=Partial, gby=[date_trunc(minute, to_timestamp_seconds(EventTime@0)) as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 08)--------------FilterExec: CounterID@2 = 62 AND EventDate@1 >= 15900 AND EventDate@1 <= 15901 AND IsRefresh@3 = 0 AND DontCountHits@4 = 0, projection=[EventTime@0] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15900 AND EventDate@5 <= 15901 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15900 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15901 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0)] @@ -1220,9 +1220,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [count(DISTINCT hits.SocialNetwork)@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(DISTINCT hits.SocialNetwork)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[BrowserCountry@0 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)] +03)----AggregateExec: mode=FinalPartitioned, gby=[BrowserCountry@0 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([BrowserCountry@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[BrowserCountry@2 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)] +05)--------AggregateExec: mode=Partial, gby=[BrowserCountry@2 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[HitColor, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction], file_type=parquet query TIIII diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index a1c0e6303a76..ffe08b7dd513 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -48,9 +48,9 @@ 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, gby=[a@0 as a], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] +04)------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -64,9 +64,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as cnt] 02)--FilterExec: count(Int64(1))@1 > 0 -03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query II diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 0659b9c208f9..1f89f6422ea4 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -97,9 +97,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -06)----------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]] +06)----------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]], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 8c055c25caeb..2788973465a2 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2021,9 +2021,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -06)----------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]] +06)----------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]], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)], projection=[col0@2, col1@3, col2@4, col0@0, col1@1] 09)----------------DataSourceExec: partitions=1, partition_sizes=[3] @@ -2987,9 +2987,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 -06)----------AggregateExec: mode=Partial, 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]] +06)----------AggregateExec: mode=Partial, 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]], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR @@ -3021,9 +3021,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 -06)----------AggregateExec: mode=Partial, 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]] +06)----------AggregateExec: mode=Partial, 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]], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3183,9 +3183,9 @@ 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, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +06)----------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]], stream=RawPartialHashAggregateStream 07)------------SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3218,9 +3218,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -06)----------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]] +06)----------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]], stream=RawPartialHashAggregateStream 07)------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[true] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3418,9 +3418,9 @@ 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, gby=[sn@0 as sn, amount@1 as amount], aggr=[], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[2] @@ -3486,9 +3486,9 @@ 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, gby=[sn@0 as sn, amount@1 as amount], aggr=[sum(l.amount)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)] +06)----------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)], stream=RawPartialHashAggregateStream 07)------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1, projection=[amount@1, sn@2, amount@3] 08)--------------DataSourceExec: partitions=1, partition_sizes=[2] 09)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -3632,9 +3632,9 @@ 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, 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=[], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 -06)----------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=[] +06)----------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=[], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 08)--------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] 09)----------------BoundedWindowAggExec: wdw=[sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": nullable Float64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] @@ -3859,9 +3859,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 -04)------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]] +04)------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]], stream=RawPartialHashAggregateStream 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1, maintains_sort_order=true 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true @@ -4226,9 +4226,9 @@ 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, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +04)------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -4241,12 +4241,12 @@ 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, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -04)------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +04)------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(x@0 AS Float64) as alias1], aggr=[] +07)------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(x@0 AS Float64) as alias1], aggr=[], stream=RawPartialHashAggregateStream 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. @@ -4440,12 +4440,12 @@ 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, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] -07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3] +06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3], stream=PartialFinalHashAggregateStream 08)--------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 -09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3] +09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3], stream=RawPartialHashAggregateStream 10)------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt index 16ab90651c8b..d789ccac3aed 100644 --- a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt +++ b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt @@ -142,26 +142,26 @@ physical_plan 01)SortPreservingMergeExec: [channel@0 ASC, brand@1 ASC] 02)--SortExec: expr=[channel@0 ASC, brand@1 ASC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[channel@0 as channel, brand@1 as brand, sum(sub.total)@3 as grand_total] -04)------AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, brand@1 as brand, __grouping_id@2 as __grouping_id], aggr=[sum(sub.total)] +04)------AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, brand@1 as brand, __grouping_id@2 as __grouping_id], aggr=[sum(sub.total)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([channel@0, brand@1, __grouping_id@2], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as brand), (channel@0 as channel, NULL as brand), (channel@0 as channel, brand@1 as brand)], aggr=[sum(sub.total)] 07)------------InterleaveExec 08)--------------ProjectionExec: expr=[store as channel, brand@0 as brand, sum(sales.amount)@1 as total] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=PartialFinalHashAggregateStream 10)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] +11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=RawPartialHashAggregateStream 12)----------------------FilterExec: channel@0 = store, projection=[brand@1, amount@2] 13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] 14)--------------ProjectionExec: expr=[web as channel, brand@0 as brand, sum(sales.amount)@1 as total] -15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] +15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=PartialFinalHashAggregateStream 16)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 -17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] +17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=RawPartialHashAggregateStream 18)----------------------FilterExec: channel@0 = web, projection=[brand@1, amount@2] 19)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] 20)--------------ProjectionExec: expr=[catalog as channel, brand@0 as brand, sum(sales.amount)@1 as total] -21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] +21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=PartialFinalHashAggregateStream 22)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 -23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] +23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=RawPartialHashAggregateStream 24)----------------------FilterExec: channel@0 = catalog, projection=[brand@1, amount@2] 25)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 3bf101f203fb..c9b3374bd267 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -481,7 +481,7 @@ datafusion.optimizer.repartition_joins true Should DataFusion repartition data u datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below ```text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ``` would turn into the plan below which performs better in multithreaded environments ```text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ``` datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail -datafusion.optimizer.subset_repartition_threshold 4 Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): ```text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ``` +datafusion.optimizer.subset_repartition_threshold 4 Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): ```text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ```, stream=RawPartialHashAggregateStream 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.use_statistics_registry false When set to true, the physical plan optimizer uses the pluggable `StatisticsRegistry` for statistics propagation across operators. This enables more accurate cardinality estimates compared to each operator's built-in `partition_statistics`. datafusion.runtime.file_statistics_cache_limit 20M Maximum memory to use for file statistics cache. Supports suffixes K (kilobytes), M (megabytes), and G (gigabytes) or '0' for 0. Example: '2G' for 2 gigabytes. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index b9d163d87759..26b062587da7 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1422,9 +1422,9 @@ physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[v0@2, v1@3, sum(t1.v1)@0] 02)--CoalescePartitionsExec 03)----ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] -04)------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +06)----------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)], stream=RawPartialHashAggregateStream 07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index e0be63fe7152..fce9f7f8f360 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1338,9 +1338,9 @@ 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, gby=[t1_id@0 as t1_id], aggr=[], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] +03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[], stream=RawPartialHashAggregateStream 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1393,9 +1393,9 @@ 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, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1420,9 +1420,9 @@ 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, gby=[alias1@0 as alias1], aggr=[], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] +07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/lateral_join.slt b/datafusion/sqllogictest/test_files/lateral_join.slt index cae3e6715324..5bb11e767466 100644 --- a/datafusion/sqllogictest/test_files/lateral_join.slt +++ b/datafusion/sqllogictest/test_files/lateral_join.slt @@ -773,9 +773,9 @@ physical_plan 04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, __always_true@3, cnt@1] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------ProjectionExec: expr=[count(Int64(1))@1 as cnt, t1_id@0 as t1_id, true as __always_true] -07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 08)------------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=1 -09)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] +09)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 10)----------------DataSourceExec: partitions=1, partition_sizes=[1] # Verify LEFT lateral without aggregate decorrelates to left join diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ca2b36727d62..4f98fa4fc90e 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -422,9 +422,9 @@ 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, gby=[i@0 as i], aggr=[], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=1 -03)----AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] +03)----AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[], stream=RawPartialHashAggregateStream 04)------DataSourceExec: partitions=1 statement ok @@ -652,9 +652,9 @@ 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, gby=[b@0 as b], aggr=[sum(ordered_table.a)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] +06)----------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true @@ -1037,9 +1037,9 @@ physical_plan 03)----SortPreservingMergeExec: [sx@1 DESC], fetch=4 04)------SortExec: TopK(fetch=4), expr=[sx@1 DESC], preserve_partitioning=[true] 05)--------ProjectionExec: expr=[g@0 as g, sum(t22489.x)@1 as sx, sum(t22489.y)@2 as sy] -06)----------AggregateExec: mode=FinalPartitioned, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)] +06)----------AggregateExec: mode=FinalPartitioned, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)], stream=PartialFinalHashAggregateStream 07)------------RepartitionExec: partitioning=Hash([g@0], 4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)] +08)--------------AggregateExec: mode=Partial, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)], stream=RawPartialHashAggregateStream 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 6907e489e690..ccbf179cfaab 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -274,9 +274,9 @@ physical_plan 01)SortPreservingMergeExec: [c2@0 ASC NULLS LAST] 02)--SortExec: expr=[c2@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -293,9 +293,9 @@ physical_plan 01)SortPreservingMergeExec: [total_sal@1 ASC NULLS LAST, c2@0 ASC NULLS LAST] 02)--SortExec: expr=[total_sal@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -312,9 +312,9 @@ physical_plan 01)SortPreservingMergeExec: [c2@0 ASC NULLS LAST, abs(c2@0) ASC NULLS LAST] 02)--SortExec: expr=[c2@0 ASC NULLS LAST, abs(c2@0) ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -1034,15 +1034,15 @@ 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, gby=[t@0 as t], aggr=[], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +07)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[], stream=RawPartialHashAggregateStream 08)--------------ProjectionExec: expr=[column1@0 as t] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------ProjectionExec: expr=[1 as m, t@0 as t] -11)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +11)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[], stream=PartialFinalHashAggregateStream 12)----------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 -13)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +13)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[], stream=RawPartialHashAggregateStream 14)--------------ProjectionExec: expr=[column1@0 as t] 15)----------------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 175d7d90cd8e..41a1021781eb 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -227,9 +227,9 @@ logical_plan 03)----TableScan: fact_table projection=[value, f_dkey] physical_plan 01)ProjectionExec: expr=[f_dkey@0 as f_dkey, count(Int64(1))@1 as count(*), sum(fact_table.value)@2 as sum(fact_table.value)] -02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] +02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] +04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet # Verify results without optimization @@ -589,9 +589,9 @@ logical_plan 03)----TableScan: high_cardinality_table projection=[value, f_dkey] physical_plan 01)ProjectionExec: expr=[f_dkey@0 as f_dkey, count(Int64(1))@1 as count(*), sum(high_cardinality_table.value)@2 as sum(high_cardinality_table.value)] -02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] +02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] +04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=E/data.parquet]]}, projection=[value, f_dkey], file_type=parquet query TIR rowsort @@ -636,9 +636,9 @@ logical_plan 06)------SubqueryAlias: d 07)--------TableScan: dimension_table_partitioned projection=[env, d_dkey] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)] +01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([f_dkey@0, env@1], 3), input_partitions=3 -03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)] +03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)], stream=RawPartialHashAggregateStream 04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[value@2, f_dkey@3, env@0] 05)--------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet @@ -681,9 +681,9 @@ logical_plan 06)------SubqueryAlias: d 07)--------TableScan: dimension_table_partitioned projection=[env, d_dkey] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)] +01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([f_dkey@0, env@1], 3), input_partitions=3 -03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)] +03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)], stream=RawPartialHashAggregateStream 04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[value@2, f_dkey@3, env@0] 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet 06)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index 344aef1f92cf..f3b474067147 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -790,9 +790,9 @@ logical_plan 04)------TableScan: multi_struct projection=[s] physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as multi_struct.s[label], sum(__datafusion_extracted_2)@1 as sum(multi_struct.s[value])] -02)--AggregateExec: mode=FinalPartitioned, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] +02)--AggregateExec: mode=FinalPartitioned, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([__datafusion_extracted_1@0], 4), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] +04)------AggregateExec: mode=Partial, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)], stream=RawPartialHashAggregateStream 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet # Verify correctness diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index 40bfe79dcc63..f2fd1089079c 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -741,9 +741,9 @@ Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)], projection=[a@0, min_value@2], metrics=[output_rows=2, output_batches=2, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_build.parquet]]}, projection=[a], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=14.45% (64/443)] 03)--ProjectionExec: expr=[a@0 as a, min(join_agg_probe.value)@1 as min_value], metrics=[output_rows=2, output_batches=2] -04)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] +04)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], stream=PartialFinalHashAggregateStream, metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] 05)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1, metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] -06)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=1, spill_count=0, spilled_rows=0, skipped_aggregation_rows=0, reduction_factor=100% (2/2)] +06)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], stream=RawPartialHashAggregateStream, metrics=[output_rows=2, output_batches=1, spill_count=0, spilled_rows=0, skipped_aggregation_rows=0, reduction_factor=100% (2/2)] 07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet]]}, projection=[a, value], file_type=parquet, predicate=DynamicFilter [ a@0 >= h1 AND a@0 <= h2 AND a@0 IN (SET) ([h1, h2]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= h1 AND a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND (a_null_count@1 != row_count@2 AND a_min@3 <= h1 AND h1 <= a_max@0 OR a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND h2 <= a_max@0), required_guarantees=[a in (h1, h2)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=4, predicate_cache_records=2, scan_efficiency_ratio=19.07% (151/792)] statement ok diff --git a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index 923a51afc8df..2cf9ef52038c 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -180,7 +180,7 @@ group by grouping sets ((), (id)) ---- physical_plan 01)ProjectionExec: expr=[max(agg_dyn_test.id)@2 as max(agg_dyn_test.id)] -02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)] +02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([id@0, __grouping_id@1], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[(NULL as id), (id@0 as id)], aggr=[max(agg_dyn_test.id)] 05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] @@ -497,9 +497,9 @@ EXPLAIN SELECT a, count(b) AS cnt FROM agg_filter_pushdown GROUP BY a HAVING cou physical_plan 01)ProjectionExec: expr=[a@0 as a, count(agg_filter_pushdown.b)@1 as cnt] 02)--FilterExec: count(agg_filter_pushdown.b)@1 > 5 -03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)] +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)], stream=PartialFinalHashAggregateStream 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)] +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)], stream=RawPartialHashAggregateStream 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_filter_pushdown.parquet]]}, projection=[a, b], file_type=parquet statement ok diff --git a/datafusion/sqllogictest/test_files/qualify.slt b/datafusion/sqllogictest/test_files/qualify.slt index 68aae16d9014..e0a68be0f238 100644 --- a/datafusion/sqllogictest/test_files/qualify.slt +++ b/datafusion/sqllogictest/test_files/qualify.slt @@ -322,9 +322,9 @@ physical_plan 05)--------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 07)------------FilterExec: sum(users.salary)@2 > 20000.00, projection=[dept@0, salary@1] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)], stream=PartialFinalHashAggregateStream 09)----------------RepartitionExec: partitioning=Hash([dept@0, salary@1], 4), input_partitions=4 -10)------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)] +10)------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)], stream=RawPartialHashAggregateStream 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------FilterExec: salary@0 > 5000.00 13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -353,9 +353,9 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 07)------------SortPreservingMergeExec: [sum(users.salary)@1 DESC] 08)--------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)], stream=PartialFinalHashAggregateStream 10)------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 -11)--------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] +11)--------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)], stream=RawPartialHashAggregateStream 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] # Clean up diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index cf913caefc52..95758642cd65 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -43,9 +43,9 @@ 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, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 -03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)], stream=RawPartialHashAggregateStream 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # disable round robin repartitioning @@ -59,9 +59,9 @@ 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, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 -03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)], stream=RawPartialHashAggregateStream 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index dbf31dec5e11..4f944075b3b0 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -369,9 +369,9 @@ physical_plan 01)SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] 02)--SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[env@0 as env, time_bin@1 as time_bin, avg(a.max_bin_value)@2 as avg_max_value] -04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)] +04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 -06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] +06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) 09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3, preserve_order=true, sort_exprs=f_dkey@0 ASC NULLS LAST, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 ASC NULLS LAST @@ -466,9 +466,9 @@ physical_plan 01)SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] 02)--SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[env@0 as env, time_bin@1 as time_bin, avg(a.max_bin_value)@2 as avg_max_value] -04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)] +04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 -06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] +06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) 09)----------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[f_dkey@4, env@0, timestamp@2, value@3] diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 762f5c11333d..d22be62d4743 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1573,9 +1573,9 @@ 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, gby=[c2@0 as c2], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 03)----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(Int64(1))] +04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[c2@0 as c2] 07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index dd195b0ff487..d292f47113c3 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -204,9 +204,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=RawPartialHashAggregateStream 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -237,9 +237,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int * Float64(1)) + Int64(1)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int * Float64(1))@1 + 1 as sum(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))] +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))], stream=RawPartialHashAggregateStream 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -270,9 +270,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=RawPartialHashAggregateStream 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -304,9 +304,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------FilterExec: sum(t2.t2_int)@1 < 3, projection=[sum(t2.t2_int)@1, t2_id@0] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=RawPartialHashAggregateStream 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index 6df93a3daabf..5ff3c2bf2ba0 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -168,9 +168,9 @@ 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, 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]], stream=PartialFinalHashAggregateStream 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -07)------------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]] +07)------------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]], stream=RawPartialHashAggregateStream 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index db4c98161c20..cca3ae69983f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -50,9 +50,9 @@ 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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 -06)----------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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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))] +06)----------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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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))], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[l_extendedprice@0 * (1 - l_discount@1) as __common_expr_1, l_quantity@2 as l_quantity, l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 08)--------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_extendedprice@1, l_discount@2, l_quantity@0, l_tax@3, l_returnflag@4, l_linestatus@5] 09)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 210468450d45..55b2a4c4ddfd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -72,9 +72,9 @@ 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 * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +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 * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] 08)--------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index e8a224867df0..475ec102ea31 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -78,9 +78,9 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[value@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] 05)--------FilterExec: CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) > scalar_subquery() -06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], stream=PartialFinalHashAggregateStream 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)] +08)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], stream=RawPartialHashAggregateStream 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] 10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index 84a6598cb992..af212701f014 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -62,9 +62,9 @@ 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 = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as 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, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)], stream=RawPartialHashAggregateStream 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 24e23e4dbd0a..2f14ee491f91 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -56,9 +56,9 @@ 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, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 08)--------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 09)----------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 5af08fa79c92..64cdc27956f2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -78,17 +78,17 @@ physical_plan 06)----------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 07)--------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] 08)----------FilterExec: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 = scalar_subquery() -09)------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream 10)--------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -11)----------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +11)----------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream 12)------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] 13)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false 14)--AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] 15)----CoalescePartitionsExec 16)------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] 17)--------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -18)----------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +18)----------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream 19)------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -20)--------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +20)--------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream 21)----------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] 22)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 0d5e0c030321..16b8a908c589 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -68,12 +68,12 @@ 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, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 -06)----------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)] -07)------------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=[] +06)----------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)], stream=RawPartialHashAggregateStream +07)------------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=[], stream=PartialFinalHashAggregateStream 08)--------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 -09)----------------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=[] +09)----------------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=[], stream=RawPartialHashAggregateStream 10)------------------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] 11)--------------------CoalescePartitionsExec 12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 9f375a583f77..40f29cfbb734 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -63,7 +63,7 @@ physical_plan 10)--------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] 11)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:0..597773], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:597773..1195546], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1195546..1793319], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1793319..2391090]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false 12)----------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] -13)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +13)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], stream=PartialFinalHashAggregateStream 14)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -15)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +15)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], stream=RawPartialHashAggregateStream 16)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 831072092b25..7fb5b1fb7846 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -81,7 +81,7 @@ physical_plan 12)----------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 13)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false 14)--------FilterExec: sum(lineitem.l_quantity)@1 > 300.00, projection=[l_orderkey@0] -15)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +15)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], stream=PartialFinalHashAggregateStream 16)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +17)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], stream=RawPartialHashAggregateStream 18)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index e471c2c23d2e..c26fd7bad2b6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -125,9 +125,9 @@ physical_plan 24)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false 25)------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 26)--------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -27)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +27)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], stream=PartialFinalHashAggregateStream 28)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -29)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +29)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], stream=RawPartialHashAggregateStream 30)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] 31)------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 32)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 76876160e2bb..09f5385d6436 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -102,8 +102,8 @@ physical_plan 19)----------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] 20)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:0..597773], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:597773..1195546], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1195546..1793319], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1793319..2391090]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false 21)----------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] -22)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +22)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], stream=PartialFinalHashAggregateStream 23)--------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -24)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +24)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], stream=RawPartialHashAggregateStream 25)------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] 26)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index 5e9192d67753..a4da51047fef 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -93,9 +93,9 @@ 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, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] 08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 09)----------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 97f017eff226..0e598dda40b5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -76,9 +76,9 @@ physical_plan 02)--SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] 03)----SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] -05)--------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], stream=PartialFinalHashAggregateStream 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)] +07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], stream=RawPartialHashAggregateStream 08)--------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] 10)------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 0007666f1536..13443eb4bfd9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -56,9 +56,9 @@ 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, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] +06)----------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream 07)------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 08)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 6cbc9c4bef26..c8759b43d658 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -70,9 +70,9 @@ 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 * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] 08)--------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 4bcb738d621d..518814605112 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -87,9 +87,9 @@ 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, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 -06)----------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)] +06)----------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)], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[n_name@0 as supp_nation, n_name@1 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@3 * (1 - l_discount@4) as volume] 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@4, n_name@6, l_shipdate@2, l_extendedprice@0, l_discount@1] 09)----------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 189d501ce207..704681551f07 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -93,9 +93,9 @@ 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 = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] +04)------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] +06)----------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@0) as o_year, l_extendedprice@1 * (1 - l_discount@2) as volume, n_name@3 as nation] 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[o_orderdate@2, l_extendedprice@0, l_discount@1, n_name@4] 09)----------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 84b8e6fffd16..2a1b17b6a5ac 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -78,9 +78,9 @@ 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, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] +06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], stream=RawPartialHashAggregateStream 07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@2 * (1 - l_discount@3) - ps_supplycost@4 * l_quantity@5 as amount] 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_extendedprice@1, l_discount@2, ps_supplycost@4, l_quantity@0] 09)----------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index a48ede604968..e052a8f537e6 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -233,9 +233,9 @@ logical_plan 05)----Projection: t2.name || Utf8View("_new") AS name 06)------TableScan: t2 projection=[name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 -03)----AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +03)----AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream 04)------UnionExec 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------DataSourceExec: partitions=1, partition_sizes=[1] @@ -313,9 +313,9 @@ logical_plan 05)----Filter: t1.id = Int32(2) 06)------TableScan: t1 projection=[id, name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 05)--------UnionExec 06)----------FilterExec: id@0 = 1 @@ -334,9 +334,9 @@ logical_plan 02)--Filter: t1.id = Int32(1) OR t1.id = Int32(2) 03)----TableScan: t1 projection=[id, name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream 02)--RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------FilterExec: id@0 = 1 OR id@0 = 2 06)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -381,18 +381,18 @@ physical_plan 04)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +07)----AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream 08)------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -09)--------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +09)--------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream 10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)------------DataSourceExec: partitions=1, partition_sizes=[1] 12)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 13)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true 14)------CoalescePartitionsExec 15)--------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -16)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +16)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream 17)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -18)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +18)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream 19)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 20)------------------DataSourceExec: partitions=1, partition_sizes=[1] 21)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -444,18 +444,18 @@ physical_plan 01)UnionExec 02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true 03)----CoalescePartitionsExec -04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------DataSourceExec: partitions=1, partition_sizes=[1] 11)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true 12)----CoalescePartitionsExec -13)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +13)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream 14)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -15)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +15)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream 16)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 17)--------------DataSourceExec: partitions=1, partition_sizes=[1] 18)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -513,14 +513,14 @@ 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, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +09)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream 10)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -11)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +11)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream 12)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 13)--------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -570,9 +570,9 @@ physical_plan 06)----------CoalescePartitionsExec 07)------------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 08)--------------ProjectionExec: expr=[] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[], stream=PartialFinalHashAggregateStream 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] +11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[], stream=RawPartialHashAggregateStream 12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 1c614f6a22c1..f02b22b61ca1 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, gby=[b@0 as b], aggr=[max(d.a)], stream=PartialFinalHashAggregateStream 05)--------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 07)------------UnionExec @@ -1877,9 +1877,9 @@ 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, gby=[c1@0 as c1], aggr=[], stream=PartialFinalHashAggregateStream 07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] +08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[], stream=RawPartialHashAggregateStream 09)----------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true From e171dac5a9ea938953549c2f2f93289e82793100 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Mon, 1 Jun 2026 12:31:52 +0800 Subject: [PATCH 3/8] draft blocked design for primitive GroupValues and avg GroupsAccumulator --- .../expr-common/src/groups_accumulator.rs | 11 +- datafusion/ffi/src/udaf/groups_accumulator.rs | 4 + .../src/aggregate/count_distinct/groups.rs | 6 +- .../groups_accumulator/accumulate.rs | 2 +- .../aggregate/groups_accumulator/bool_op.rs | 2 +- .../functions-aggregate/src/array_agg.rs | 4 +- .../functions-aggregate/src/first_last.rs | 2 +- .../src/first_last/state.rs | 2 +- .../src/min_max/min_max_bytes.rs | 2 +- .../src/min_max/min_max_struct.rs | 2 +- .../src/aggregates/group_values/mod.rs | 29 +- .../group_values/multi_group_by/mod.rs | 2 +- .../src/aggregates/group_values/row.rs | 2 +- .../single_group_by/blocked_primitive.rs | 314 ++++++++ .../group_values/single_group_by/boolean.rs | 2 +- .../group_values/single_group_by/bytes.rs | 2 +- .../single_group_by/bytes_view.rs | 2 +- .../group_values/single_group_by/mod.rs | 1 + .../group_values/single_group_by/primitive.rs | 2 +- .../physical-plan/src/aggregates/order/mod.rs | 2 +- .../src/aggregates/raw_partial_hash.rs | 681 +++++++++++++++++- 21 files changed, 1032 insertions(+), 44 deletions(-) create mode 100644 datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 9053f7a8eab9..ad67f763302b 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -25,6 +25,15 @@ use datafusion_common::{Result, not_impl_err}; pub enum EmitTo { /// Emit all groups All, + /// Emit one implementation-defined block of groups. + /// + /// This is intended for accumulators and group-value stores whose internal + /// representation is already split into fixed-size blocks. Implementations + /// without internal blocking may treat this the same as [`Self::All`]. + /// + /// Callers should only use this once no further updates will arrive for the + /// current groups. + Block, /// Emit only the first `n` groups and shift all existing group /// indexes down by `n`. /// @@ -41,7 +50,7 @@ impl EmitTo { /// This avoids copying if Self::All pub fn take_needed(&self, v: &mut Vec) -> Vec { match self { - Self::All => { + Self::All | Self::Block => { // Take the entire vector, leave new (empty) vector std::mem::take(v) } diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 1600bef39da4..92d2b5e631ad 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -450,12 +450,14 @@ impl GroupsAccumulator for ForeignGroupsAccumulator { pub enum FFI_EmitTo { All, First(usize), + Block, } impl From for FFI_EmitTo { fn from(value: EmitTo) -> Self { match value { EmitTo::All => Self::All, + EmitTo::Block => Self::Block, EmitTo::First(v) => Self::First(v), } } @@ -465,6 +467,7 @@ impl From for EmitTo { fn from(value: FFI_EmitTo) -> Self { match value { FFI_EmitTo::All => Self::All, + FFI_EmitTo::Block => Self::Block, FFI_EmitTo::First(v) => Self::First(v), } } @@ -552,6 +555,7 @@ mod tests { fn test_all_emit_to_round_trip() -> Result<()> { test_emit_to_round_trip(EmitTo::All)?; test_emit_to_round_trip(EmitTo::First(10))?; + test_emit_to_round_trip(EmitTo::Block)?; Ok(()) } diff --git a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs index d370d59c9001..a75363e59217 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs @@ -85,7 +85,7 @@ where let counts = emit_to.take_needed(&mut self.counts); match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { self.seen.clear(); } EmitTo::First(n) => { @@ -104,7 +104,7 @@ where fn state(&mut self, emit_to: EmitTo) -> datafusion_common::Result> { let num_emitted = match emit_to { - EmitTo::All => self.counts.len(), + EmitTo::All | EmitTo::Block => self.counts.len(), EmitTo::First(n) => n, }; @@ -120,7 +120,7 @@ where let mut all_values = vec![T::Native::default(); total as usize]; let mut cursors: Vec = offsets[..num_emitted].to_vec(); - if matches!(emit_to, EmitTo::All) { + if matches!(emit_to, EmitTo::All | EmitTo::Block) { for (group_idx, value) in self.seen.drain() { let pos = cursors[group_idx] as usize; all_values[pos] = value; diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 09e1df4eae70..235ad10a8a4c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -296,7 +296,7 @@ impl NullState { /// resets the internal state appropriately pub fn build(&mut self, emit_to: EmitTo) -> Option { match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { let old_seen = std::mem::take(&mut self.seen_values); match old_seen { SeenValues::All { .. } => None, diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs index d1d8924a2c3e..3929d8edac59 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs @@ -108,7 +108,7 @@ where let values = self.values.finish(); let values = match emit_to { - EmitTo::All => values, + EmitTo::All | EmitTo::Block => values, EmitTo::First(n) => { let first_n: BooleanBuffer = values.iter().take(n).collect(); // put n+1 back into self.values diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 24edaaff1f09..bf7e18722b54 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -652,7 +652,7 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { /// the values into a flat array that backs the output `ListArray`. fn evaluate(&mut self, emit_to: EmitTo) -> Result { let emit_groups = match emit_to { - EmitTo::All => self.num_groups, + EmitTo::All | EmitTo::Block => self.num_groups, EmitTo::First(n) => n, }; @@ -711,7 +711,7 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { // Step 4: Release state for emitted groups. match emit_to { - EmitTo::All => self.clear_state(), + EmitTo::All | EmitTo::Block => self.clear_state(), EmitTo::First(_) => self.compact_retained_state(emit_groups)?, } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 1935f29c4cfe..cc60c2e82b27 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -441,7 +441,7 @@ impl FirstLastGroupsAccumulator { let result = emit_to.take_needed(&mut self.orderings); match emit_to { - EmitTo::All => self.size_of_orderings = 0, + EmitTo::All | EmitTo::Block => self.size_of_orderings = 0, EmitTo::First(_) => { self.size_of_orderings -= result.iter().map(ScalarValue::size_of_vec).sum::() diff --git a/datafusion/functions-aggregate/src/first_last/state.rs b/datafusion/functions-aggregate/src/first_last/state.rs index cd7114bf04f9..e83b190468b7 100644 --- a/datafusion/functions-aggregate/src/first_last/state.rs +++ b/datafusion/functions-aggregate/src/first_last/state.rs @@ -296,7 +296,7 @@ pub(crate) fn take_need( ) -> BooleanBuffer { let bool_buf = bool_buf_builder.finish(); match emit_to { - EmitTo::All => bool_buf, + EmitTo::All | EmitTo::Block => bool_buf, EmitTo::First(n) => { // split off the first N values in seen_values // diff --git a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs index b56c2106e32b..adcec89537ef 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -488,7 +488,7 @@ impl MinMaxBytesState { /// - `min_maxes`: the actual min/max values for each group fn emit_to(&mut self, emit_to: EmitTo) -> (usize, Vec>>) { match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { ( std::mem::take(&mut self.total_data_bytes), // reset total bytes and min_max std::mem::take(&mut self.min_max), diff --git a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs index 7c94e7f5738b..ec9d9e62b4b1 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs @@ -277,7 +277,7 @@ impl MinMaxStructState { /// - `min_maxes`: the actual min/max values for each group fn emit_to(&mut self, emit_to: EmitTo) -> (usize, Vec>) { match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { ( std::mem::take(&mut self.total_data_bytes), // reset total bytes and min_max std::mem::take(&mut self.min_max), diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 3cbfa003ab2e..3b3a0b6b90f0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -18,9 +18,10 @@ //! [`GroupValues`] trait for storing and interning group keys use arrow::array::types::{ - Date32Type, Date64Type, Decimal128Type, Time32MillisecondType, Time32SecondType, - Time64MicrosecondType, Time64NanosecondType, TimestampMicrosecondType, - TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, + Date32Type, Date64Type, Decimal128Type, Int64Type, Time32MillisecondType, + Time32SecondType, Time64MicrosecondType, Time64NanosecondType, + TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, + TimestampSecondType, }; use arrow::array::{ArrayRef, downcast_primitive}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; @@ -40,8 +41,9 @@ pub(crate) use single_group_by::primitive::HashValue; use crate::aggregates::{ group_values::single_group_by::{ - boolean::GroupValuesBoolean, bytes::GroupValuesBytes, - bytes_view::GroupValuesBytesView, primitive::GroupValuesPrimitive, + blocked_primitive::GroupValuesPrimitiveBlock, boolean::GroupValuesBoolean, + bytes::GroupValuesBytes, bytes_view::GroupValuesBytesView, + primitive::GroupValuesPrimitive, }, order::GroupOrdering, }; @@ -143,6 +145,23 @@ pub fn new_unordered_group_values(schema: SchemaRef) -> Result Result>> { + if schema.fields.len() == 1 && matches!(schema.fields[0].data_type(), DataType::Int64) + { + return Ok(Some(Box::new(GroupValuesPrimitiveBlock::::new( + schema.fields[0].data_type().clone(), + block_size, + )))); + } + + Ok(None) +} + fn new_group_values_with_ordering( schema: SchemaRef, ordered: bool, diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs index ee2d300d9bff..876cea13f008 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs @@ -1090,7 +1090,7 @@ impl GroupValues for GroupValuesColumn { fn emit(&mut self, emit_to: EmitTo) -> Result> { let mut output = match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { let group_values = mem::take(&mut self.group_values); debug_assert!(self.group_values.is_empty()); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index a3bd31f76c23..d45a0670d14c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -202,7 +202,7 @@ impl GroupValues for GroupValuesRows { .expect("Can not emit from empty rows"); let mut output = match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { let output = self.row_converter.convert_rows(&group_values)?; group_values.clear(); self.map.clear(); diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs new file mode 100644 index 000000000000..55d7fab7b940 --- /dev/null +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs @@ -0,0 +1,314 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::aggregates::group_values::GroupValues; +use crate::aggregates::group_values::single_group_by::primitive::HashValue; + +use arrow::array::{ + ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, NullBufferBuilder, PrimitiveArray, + cast::AsArray, +}; +use arrow::datatypes::DataType; +use datafusion_common::Result; +use datafusion_common::hash_utils::RandomState; +use datafusion_execution::memory_pool::proxy::VecAllocExt; +use datafusion_expr::EmitTo; +use hashbrown::hash_table::HashTable; +use std::mem::size_of; +use std::sync::Arc; + +/// A single-column primitive [`GroupValues`] that stores group keys in fixed +/// sized blocks. Group ids are still logically contiguous: +/// +/// ```text +/// group_id = block_index * block_size + offset_in_block +/// ``` +/// +/// `EmitTo::Block` consumes exactly one stored block, so callers that configure +/// accumulators with the same block size can build output batches without first +/// materializing one large contiguous values vector. +pub struct GroupValuesPrimitiveBlock { + data_type: DataType, + map: HashTable<(usize, u64)>, + null_group: Option, + blocks: Vec>, + len: usize, + block_size: usize, + random_state: RandomState, +} + +impl GroupValuesPrimitiveBlock { + pub fn new(data_type: DataType, block_size: usize) -> Self { + assert!(block_size > 0); + assert!(PrimitiveArray::::is_compatible(&data_type)); + Self { + data_type, + map: HashTable::with_capacity(128), + blocks: Vec::with_capacity(1), + null_group: None, + len: 0, + block_size, + random_state: crate::aggregates::AGGREGATION_HASH_SEED, + } + } + + fn push_value(&mut self, value: T::Native) -> usize + where + T::Native: Default + Copy, + { + let group_id = self.len; + if group_id.is_multiple_of(self.block_size) { + self.blocks + .push(vec![T::Native::default(); self.block_size].into_boxed_slice()); + } + + let block_idx = group_id / self.block_size; + let value_idx = group_id % self.block_size; + self.blocks[block_idx][value_idx] = value; + self.len += 1; + group_id + } + + fn release_map(&mut self) { + self.map.clear(); + self.map.shrink_to(0, |_| 0); + } + + fn nulls_for( + null_idx: Option, + len: usize, + ) -> Option { + null_idx.map(|null_idx| { + let mut buffer = NullBufferBuilder::new(len); + buffer.append_n_non_nulls(null_idx); + buffer.append_null(); + buffer.append_n_non_nulls(len - null_idx - 1); + buffer.finish().unwrap() + }) + } + + fn build_array( + data_type: &DataType, + values: Vec, + null_idx: Option, + ) -> ArrayRef { + let nulls = Self::nulls_for(null_idx, values.len()); + Arc::new( + PrimitiveArray::::new(values.into(), nulls) + .with_data_type(data_type.clone()), + ) + } + + fn take_null_for_emit(&mut self, emit_len: usize) -> Option { + match self.null_group { + Some(null_group) if null_group < emit_len => { + self.null_group = None; + Some(null_group) + } + Some(null_group) => { + self.null_group = Some(null_group - emit_len); + None + } + None => None, + } + } + + fn values_range(&self, start: usize, len: usize) -> Vec + where + T::Native: Copy, + { + let mut output = Vec::with_capacity(len); + let mut remaining = len; + let mut group_id = start; + + while remaining > 0 { + let block_idx = group_id / self.block_size; + let offset = group_id % self.block_size; + let take = remaining.min(self.block_size - offset); + output.extend_from_slice(&self.blocks[block_idx][offset..offset + take]); + remaining -= take; + group_id += take; + } + + output + } + + fn rebuild_from_values(&mut self, values: &[T::Native]) + where + T::Native: Default + Copy, + { + self.blocks.clear(); + self.len = 0; + + for chunk in values.chunks(self.block_size) { + let mut block = + vec![T::Native::default(); self.block_size].into_boxed_slice(); + block[..chunk.len()].copy_from_slice(chunk); + self.blocks.push(block); + self.len += chunk.len(); + } + } + + fn take_all_values(&mut self) -> Vec + where + T::Native: Copy, + { + let output = self.values_range(0, self.len); + self.blocks.clear(); + self.len = 0; + output + } + + fn take_block_values(&mut self) -> (Vec, Option) { + self.release_map(); + + let emit_len = self.len.min(self.block_size); + let block = self.blocks.remove(0); + let mut values = block.into_vec(); + values.truncate(emit_len); + self.len -= emit_len; + let null_idx = self.take_null_for_emit(emit_len); + + (values, null_idx) + } +} + +impl GroupValues for GroupValuesPrimitiveBlock +where + T: ArrowPrimitiveType, + T::Native: HashValue + Default + Copy + ArrowNativeTypeOp, +{ + fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { + assert_eq!(cols.len(), 1); + groups.clear(); + + for value in cols[0].as_primitive::() { + let group_id = match value { + None => match self.null_group { + Some(group_id) => group_id, + None => { + let group_id = self.push_value(T::Native::default()); + self.null_group = Some(group_id); + group_id + } + }, + Some(key) => { + let state = &self.random_state; + let hash = key.hash(state); + let blocks = self.blocks.as_ptr(); + let block_size = self.block_size; + let insert = self.map.entry( + hash, + |&(group_id, existing_hash)| { + // SAFETY: entries in `map` are created from group ids + // already stored in `blocks`, and `block_size` is fixed. + unsafe { + let block = &*blocks.add(group_id / block_size); + hash == existing_hash + && block + .get_unchecked(group_id % block_size) + .is_eq(key) + } + }, + |&(_, existing_hash)| existing_hash, + ); + + match insert { + hashbrown::hash_table::Entry::Occupied(entry) => entry.get().0, + hashbrown::hash_table::Entry::Vacant(entry) => { + let group_id = self.len; + entry.insert((group_id, hash)); + if group_id.is_multiple_of(self.block_size) { + self.blocks.push( + vec![T::Native::default(); self.block_size] + .into_boxed_slice(), + ); + } + + let block_idx = group_id / self.block_size; + let value_idx = group_id % self.block_size; + self.blocks[block_idx][value_idx] = key; + self.len += 1; + group_id + } + } + } + }; + groups.push(group_id); + } + + Ok(()) + } + + fn size(&self) -> usize { + self.map.capacity() * size_of::<(usize, u64)>() + + self.blocks.allocated_size() + + self.blocks.len() * self.block_size * size_of::() + } + + fn is_empty(&self) -> bool { + self.len == 0 + } + + fn len(&self) -> usize { + self.len + } + + fn emit(&mut self, emit_to: EmitTo) -> Result> { + let data_type = self.data_type.clone(); + let output = match emit_to { + EmitTo::All => { + self.release_map(); + let null_idx = self.null_group.take(); + let values = self.take_all_values(); + Self::build_array(&data_type, values, null_idx) + } + EmitTo::Block => { + let (values, null_idx) = self.take_block_values(); + Self::build_array(&data_type, values, null_idx) + } + EmitTo::First(n) => { + let n = n.min(self.len); + let null_idx = self.take_null_for_emit(n); + let output = self.values_range(0, n); + let remaining = self.values_range(n, self.len - n); + + self.map.retain(|entry| match entry.0.checked_sub(n) { + Some(group_id) => { + entry.0 = group_id; + true + } + None => false, + }); + self.rebuild_from_values(&remaining); + + Self::build_array(&data_type, output, null_idx) + } + }; + + Ok(vec![output]) + } + + fn clear_shrink(&mut self, num_rows: usize) { + self.blocks.clear(); + self.blocks.shrink_to(num_rows.div_ceil(self.block_size)); + self.len = 0; + self.null_group = None; + self.map.clear(); + self.map.shrink_to(num_rows, |_| 0); + } +} diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs index e993c0c53d19..2d599f9c394c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs @@ -101,7 +101,7 @@ impl GroupValues for GroupValuesBoolean { let len = self.len(); let mut builder = BooleanBufferBuilder::new(len); let emit_count = match emit_to { - EmitTo::All => len, + EmitTo::All | EmitTo::Block => len, EmitTo::First(n) => n, }; builder.append_n(emit_count, false); diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs index b881a51b2547..9ee6444855e4 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs @@ -89,7 +89,7 @@ impl GroupValues for GroupValuesBytes { let map_contents = self.map.take().into_state(); let group_values = match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { self.num_groups -= map_contents.len(); map_contents } diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs index 7a56f7c52c11..ff43977ca540 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs @@ -91,7 +91,7 @@ impl GroupValues for GroupValuesBytesView { let map_contents = self.map.take().into_state(); let group_values = match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { self.num_groups -= map_contents.len(); map_contents } diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/mod.rs index 89c6b624e8e0..6381fc50cb5b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/mod.rs @@ -17,6 +17,7 @@ //! `GroupValues` implementations for single group by cases +pub(crate) mod blocked_primitive; pub(crate) mod boolean; pub(crate) mod bytes; pub(crate) mod bytes_view; diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs index efaf7eba0f1b..0e7b77e2c639 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs @@ -181,7 +181,7 @@ where } let array: PrimitiveArray = match emit_to { - EmitTo::All => { + EmitTo::All | EmitTo::Block => { self.map.clear(); build_primitive(std::mem::take(&mut self.values), self.null_group.take()) } diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 97fbd519c825..9a23ec12ddb3 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -78,7 +78,7 @@ impl GroupOrdering { GroupOrdering::Partial(_) | GroupOrdering::Full(_) => { self.emit_to().map(|emit_to| match emit_to { EmitTo::First(max) => EmitTo::First(n.min(max)), - EmitTo::All => EmitTo::First(n), + EmitTo::All | EmitTo::Block => EmitTo::First(n), }) } } diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs index 52f8b463bff4..d1bdb8d1f380 100644 --- a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -30,21 +30,30 @@ use std::collections::HashMap; use std::marker::PhantomData; +use std::mem::size_of; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow::array::{ArrayRef, AsArray, BooleanArray, new_null_array}; -use arrow::datatypes::SchemaRef; +use arrow::array::types::{Float64Type, Int64Type, UInt64Type}; +use arrow::array::{ + Array, ArrayRef, AsArray, BooleanArray, Float64Array, Int64Array, NullBufferBuilder, + UInt64Array, new_null_array, +}; +use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use futures::ready; use futures::stream::{Stream, StreamExt}; -use super::group_values::{GroupByMetrics, GroupValues, new_unordered_group_values}; +use super::group_values::{ + GroupByMetrics, GroupValues, new_unordered_blocked_group_values, + new_unordered_group_values, +}; use super::row_hash::create_group_accumulator; use super::{ AggregateExec, PhysicalGroupBy, aggregate_expressions, evaluate_group_by, @@ -92,6 +101,387 @@ struct EvaluatedAggregateBatch { accumulator_args: Vec, } +#[derive(Debug)] +struct BlockedAvgGroupsAccumulator { + state_layout: AvgStateLayout, + block_size: usize, + counts: Vec>, + sums: Vec>, + len: usize, +} + +#[derive(Debug, Clone, Copy)] +enum AvgStateLayout { + CountSumUInt64, + SumCountInt64, +} + +impl BlockedAvgGroupsAccumulator { + fn new(state_layout: AvgStateLayout, block_size: usize) -> Self { + assert!(block_size > 0); + Self { + state_layout, + block_size, + counts: vec![], + sums: vec![], + len: 0, + } + } + + fn ensure_capacity(&mut self, total_num_groups: usize) { + let required_blocks = total_num_groups.div_ceil(self.block_size); + while self.counts.len() < required_blocks { + self.counts + .push(vec![0; self.block_size].into_boxed_slice()); + self.sums + .push(vec![0.0; self.block_size].into_boxed_slice()); + } + self.len = self.len.max(total_num_groups); + } + + fn add_value(&mut self, group_index: usize, value: f64) { + debug_assert!(group_index < self.len); + let block_idx = group_index / self.block_size; + let value_idx = group_index % self.block_size; + self.counts[block_idx][value_idx] += 1; + self.sums[block_idx][value_idx] += value; + } + + fn merge_value(&mut self, group_index: usize, count: u64, sum: f64) { + debug_assert!(group_index < self.len); + let block_idx = group_index / self.block_size; + let value_idx = group_index % self.block_size; + self.counts[block_idx][value_idx] += count; + self.sums[block_idx][value_idx] += sum; + } + + fn values_range( + blocks: &[Box<[T]>], + block_size: usize, + start: usize, + len: usize, + ) -> Vec { + let mut output = Vec::with_capacity(len); + let mut remaining = len; + let mut group_id = start; + + while remaining > 0 { + let block_idx = group_id / block_size; + let offset = group_id % block_size; + let take = remaining.min(block_size - offset); + output.extend_from_slice(&blocks[block_idx][offset..offset + take]); + remaining -= take; + group_id += take; + } + + output + } + + fn rebuild_from_values(&mut self, counts: &[u64], sums: &[f64]) { + debug_assert_eq!(counts.len(), sums.len()); + self.counts.clear(); + self.sums.clear(); + self.len = counts.len(); + + for chunk in counts.chunks(self.block_size) { + let mut block = vec![0; self.block_size].into_boxed_slice(); + block[..chunk.len()].copy_from_slice(chunk); + self.counts.push(block); + } + + for chunk in sums.chunks(self.block_size) { + let mut block = vec![0.0; self.block_size].into_boxed_slice(); + block[..chunk.len()].copy_from_slice(chunk); + self.sums.push(block); + } + } + + fn take_all_values(&mut self) -> (Vec, Vec) { + let counts = Self::values_range(&self.counts, self.block_size, 0, self.len); + let sums = Self::values_range(&self.sums, self.block_size, 0, self.len); + self.counts.clear(); + self.sums.clear(); + self.len = 0; + (counts, sums) + } + + fn take_block_values(&mut self) -> (Vec, Vec) { + let emit_len = self.len.min(self.block_size); + + let mut counts = self.counts.remove(0).into_vec(); + counts.truncate(emit_len); + + let mut sums = self.sums.remove(0).into_vec(); + sums.truncate(emit_len); + + self.len -= emit_len; + (counts, sums) + } + + fn take_first_values(&mut self, n: usize) -> (Vec, Vec) { + let n = n.min(self.len); + let counts = Self::values_range(&self.counts, self.block_size, 0, n); + let sums = Self::values_range(&self.sums, self.block_size, 0, n); + + let remaining_counts = + Self::values_range(&self.counts, self.block_size, n, self.len - n); + let remaining_sums = + Self::values_range(&self.sums, self.block_size, n, self.len - n); + self.rebuild_from_values(&remaining_counts, &remaining_sums); + + (counts, sums) + } + + fn take_values(&mut self, emit_to: EmitTo) -> (Vec, Vec) { + match emit_to { + EmitTo::All => self.take_all_values(), + EmitTo::Block => self.take_block_values(), + EmitTo::First(n) => self.take_first_values(n), + } + } + + fn nulls_for_counts(counts: &[u64]) -> Option { + let mut nulls = NullBufferBuilder::new(counts.len()); + for count in counts { + if *count == 0 { + nulls.append_null(); + } else { + nulls.append_non_null(); + } + } + nulls.finish() + } + + fn filter_is_valid(opt_filter: Option<&BooleanArray>, row: usize) -> bool { + opt_filter + .map(|filter| filter.is_valid(row) && filter.value(row)) + .unwrap_or(true) + } + + fn count_state_i64(counts: &[u64]) -> Vec { + counts.iter().map(|count| *count as i64).collect() + } +} + +impl GroupsAccumulator for BlockedAvgGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + self.ensure_capacity(total_num_groups); + + if values.null_count() == 0 && opt_filter.is_none() { + for (&group_index, &value) in group_indices.iter().zip(values.values().iter()) + { + self.add_value(group_index, value); + } + } else { + for (row, &group_index) in group_indices.iter().enumerate() { + if !Self::filter_is_valid(opt_filter, row) || values.is_null(row) { + continue; + } + self.add_value(group_index, values.value(row)); + } + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let (counts, sums) = self.take_values(emit_to); + let mut values = Vec::with_capacity(counts.len()); + let mut nulls = NullBufferBuilder::new(counts.len()); + + for (count, sum) in counts.into_iter().zip(sums) { + if count == 0 { + values.push(0.0); + nulls.append_null(); + } else { + values.push(sum / count as f64); + nulls.append_non_null(); + } + } + + Ok(Arc::new(Float64Array::new(values.into(), nulls.finish()))) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + let (counts, sums) = self.take_values(emit_to); + let nulls = Self::nulls_for_counts(&counts); + + match self.state_layout { + AvgStateLayout::CountSumUInt64 => Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, + Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, + ]), + AvgStateLayout::SumCountInt64 => Ok(vec![ + Arc::new(Float64Array::new(sums.into(), nulls.clone())) as ArrayRef, + Arc::new(Int64Array::new( + Self::count_state_i64(&counts).into(), + nulls, + )) as ArrayRef, + ]), + } + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + self.ensure_capacity(total_num_groups); + + match self.state_layout { + AvgStateLayout::CountSumUInt64 => { + let partial_counts = values[0].as_primitive::(); + let partial_sums = values[1].as_primitive::(); + + for (row, &group_index) in group_indices.iter().enumerate() { + if !Self::filter_is_valid(opt_filter, row) + || partial_counts.is_null(row) + || partial_sums.is_null(row) + { + continue; + } + + self.merge_value( + group_index, + partial_counts.value(row), + partial_sums.value(row), + ); + } + } + AvgStateLayout::SumCountInt64 => { + let partial_sums = values[0].as_primitive::(); + let partial_counts = values[1].as_primitive::(); + + for (row, &group_index) in group_indices.iter().enumerate() { + if !Self::filter_is_valid(opt_filter, row) + || partial_counts.is_null(row) + || partial_sums.is_null(row) + { + continue; + } + + self.merge_value( + group_index, + partial_counts.value(row) as u64, + partial_sums.value(row), + ); + } + } + } + + Ok(()) + } + + fn convert_to_state( + &self, + values: &[ArrayRef], + opt_filter: Option<&BooleanArray>, + ) -> Result> { + assert_eq!(values.len(), 1, "single argument to convert_to_state"); + let values = values[0].as_primitive::(); + let mut counts = Vec::with_capacity(values.len()); + let mut sums = Vec::with_capacity(values.len()); + let mut nulls = NullBufferBuilder::new(values.len()); + + for row in 0..values.len() { + if Self::filter_is_valid(opt_filter, row) && values.is_valid(row) { + counts.push(1); + sums.push(values.value(row)); + nulls.append_non_null(); + } else { + counts.push(0); + sums.push(0.0); + nulls.append_null(); + } + } + + let nulls = nulls.finish(); + match self.state_layout { + AvgStateLayout::CountSumUInt64 => Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, + Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, + ]), + AvgStateLayout::SumCountInt64 => Ok(vec![ + Arc::new(Float64Array::new(sums.into(), nulls.clone())) as ArrayRef, + Arc::new(Int64Array::new( + Self::count_state_i64(&counts).into(), + nulls, + )) as ArrayRef, + ]), + } + } + + fn supports_convert_to_state(&self) -> bool { + true + } + + fn size(&self) -> usize { + self.counts.len() * self.block_size * size_of::() + + self.sums.len() * self.block_size * size_of::() + + self.counts.allocated_size() + + self.sums.allocated_size() + } +} + +fn create_blocked_group_accumulator( + agg_expr: &Arc, + block_size: usize, +) -> Result>> { + if agg_expr.fun().name() != "avg" || agg_expr.is_distinct() { + return Ok(None); + } + + let state_fields = agg_expr.state_fields()?; + let state_layout = match state_fields.as_slice() { + [count, sum] + if matches!(count.data_type(), DataType::UInt64) + && matches!(sum.data_type(), DataType::Float64) + && matches!(agg_expr.field().data_type(), DataType::Float64) => + { + Some(AvgStateLayout::CountSumUInt64) + } + [sum, count] + if matches!(sum.data_type(), DataType::Float64) + && matches!(count.data_type(), DataType::Int64) + && matches!(agg_expr.field().data_type(), DataType::Float64) => + { + Some(AvgStateLayout::SumCountInt64) + } + _ => None, + }; + + Ok(state_layout.map(|state_layout| { + Box::new(BlockedAvgGroupsAccumulator::new(state_layout, block_size)) + as Box + })) +} + +fn create_blocked_group_accumulators( + aggr_expr: &[Arc], + block_size: usize, +) -> Result>>> { + let mut accumulators = Vec::with_capacity(aggr_expr.len()); + for agg_expr in aggr_expr { + let Some(accumulator) = create_blocked_group_accumulator(agg_expr, block_size)? + else { + return Ok(None); + }; + accumulators.push(accumulator); + } + Ok(Some(accumulators)) +} + impl HashAggregateAccumulator { fn new( arguments: Vec>, @@ -209,6 +599,10 @@ struct AggregateHashTable { /// Maximum rows per emitted output batch. batch_size: usize, + /// True when group values and all accumulators use the same internal block + /// size and can emit one block per output batch. + blocked_output: bool, + /// GROUP BY expressions evaluated for each input batch. group_by: Arc, @@ -253,29 +647,53 @@ impl AggregateHashTable { &agg.mode, agg.group_by.num_group_exprs(), )?; - let accumulators: Vec<_> = agg - .aggr_expr - .iter() - .zip(aggregate_arguments) - .zip(filters) - .map(|((agg_expr, arguments), filter)| { - let accumulator = create_group_accumulator(agg_expr)?; - Ok(HashAggregateAccumulator::new( - arguments, - filter, - accumulator, - )) - }) - .collect::>()?; let group_schema = agg.group_by.group_schema(&input_schema)?; - let group_values = new_unordered_group_values(group_schema)?; + let can_use_blocked_output = + !agg.group_by.has_grouping_set() && agg.group_by.groups().len() == 1; + + let blocked_group_values = if can_use_blocked_output { + new_unordered_blocked_group_values(&group_schema, batch_size)? + } else { + None + }; + let blocked_accumulators = if can_use_blocked_output { + create_blocked_group_accumulators(&agg.aggr_expr, batch_size)? + } else { + None + }; + + let (group_values, accumulator_impls, blocked_output) = + match (blocked_group_values, blocked_accumulators) { + (Some(group_values), Some(accumulators)) => { + (group_values, accumulators, true) + } + _ => { + let group_values = new_unordered_group_values(group_schema)?; + let accumulators = agg + .aggr_expr + .iter() + .map(create_group_accumulator) + .collect::>>()?; + (group_values, accumulators, false) + } + }; + + let accumulators: Vec<_> = aggregate_arguments + .into_iter() + .zip(filters) + .zip(accumulator_impls) + .map(|((arguments, filter), accumulator)| { + HashAggregateAccumulator::new(arguments, filter, accumulator) + }) + .collect(); Ok(Self { group_by_metrics: GroupByMetrics::new(&agg.metrics, partition), input_schema, output_schema, batch_size, + blocked_output, group_by: Arc::clone(&agg.group_by), group_values, batch_group_indices: Default::default(), @@ -346,6 +764,24 @@ impl AggregateHashTable { Ok(Some(output)) } + fn next_blocked_output_batch_from( + &mut self, + build_output_batch: impl FnOnce(&mut Self) -> Result>, + ) -> Result> { + debug_assert!(self.blocked_output); + + if self.output_finished { + return Ok(None); + } + + let output = build_output_batch(self)?; + if output.is_none() { + self.output_finished = true; + } + + Ok(output) + } + fn memory_size(&self) -> usize { let acc = self .accumulators @@ -425,7 +861,11 @@ impl AggregateHashTable { } fn next_output_batch(&mut self) -> Result> { - self.next_output_batch_from(Self::build_output_batch) + if self.blocked_output { + self.next_blocked_output_batch_from(Self::build_blocked_output_batch) + } else { + self.next_output_batch_from(Self::build_output_batch) + } } fn build_output_batch(&mut self) -> Result> { @@ -448,6 +888,27 @@ impl AggregateHashTable { Ok(Some(batch)) } + fn build_blocked_output_batch(&mut self) -> Result> { + self.init_empty_grouping_sets()?; + + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::Block)?; + + for acc in self.accumulators.iter_mut() { + output.extend(acc.state(EmitTo::Block)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + debug_assert!(batch.num_rows() <= self.batch_size); + drop(timer); + Ok(Some(batch)) + } + fn init_empty_grouping_sets(&mut self) -> Result<()> { if !self.group_by.has_grouping_set() || !self.group_values.is_empty() { return Ok(()); @@ -541,7 +1002,11 @@ impl AggregateHashTable { } fn next_output_batch(&mut self) -> Result> { - self.next_output_batch_from(Self::build_output_batch) + if self.blocked_output { + self.next_blocked_output_batch_from(Self::build_blocked_output_batch) + } else { + self.next_output_batch_from(Self::build_output_batch) + } } fn build_output_batch(&mut self) -> Result> { @@ -561,6 +1026,25 @@ impl AggregateHashTable { drop(timer); Ok(Some(batch)) } + + fn build_blocked_output_batch(&mut self) -> Result> { + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::Block)?; + + for acc in self.accumulators.iter_mut() { + output.push(acc.evaluate_final(EmitTo::Block)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + debug_assert!(batch.num_rows() <= self.batch_size); + drop(timer); + Ok(Some(batch)) + } } /// Hash aggregate stream for grouped `AggregateMode::Partial`. @@ -876,3 +1360,160 @@ impl RecordBatchStream for PartialFinalHashAggregateStream { Arc::clone(&self.schema) } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::Int64Array; + use arrow::array::types::Int64Type; + use arrow::datatypes::{Field, Schema}; + + #[test] + fn blocked_group_values_emit_blocks() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "user_id", + DataType::Int64, + false, + )])); + let mut group_values = + new_unordered_blocked_group_values(&schema, 3)?.expect("blocked Int64"); + let values: ArrayRef = + Arc::new(Int64Array::from(vec![10, 20, 30, 40, 50, 60, 70])); + let mut groups = vec![]; + + group_values.intern(&[values], &mut groups)?; + + assert_eq!(groups, vec![0, 1, 2, 3, 4, 5, 6]); + assert_eq!(group_values.len(), 7); + + let block = group_values.emit(EmitTo::Block)?; + assert_eq!( + block[0].as_primitive::().values().as_ref(), + &[10, 20, 30] + ); + + let block = group_values.emit(EmitTo::Block)?; + assert_eq!( + block[0].as_primitive::().values().as_ref(), + &[40, 50, 60] + ); + + let block = group_values.emit(EmitTo::Block)?; + assert_eq!( + block[0].as_primitive::().values().as_ref(), + &[70] + ); + assert!(group_values.is_empty()); + + Ok(()) + } + + #[test] + fn blocked_avg_state_emit_blocks() -> Result<()> { + let mut accumulator = + BlockedAvgGroupsAccumulator::new(AvgStateLayout::CountSumUInt64, 3); + let values: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(1.0), + Some(2.0), + None, + Some(4.0), + Some(5.0), + Some(6.0), + Some(7.0), + ])); + let group_indices = vec![0, 1, 2, 3, 4, 5, 6]; + + accumulator.update_batch(&[values], &group_indices, None, 7)?; + + let state = accumulator.state(EmitTo::Block)?; + let counts = state[0].as_primitive::(); + let sums = state[1].as_primitive::(); + assert_eq!(counts.values().as_ref(), &[1, 1, 0]); + assert_eq!(sums.values().as_ref(), &[1.0, 2.0, 0.0]); + assert!(counts.is_null(2)); + assert!(sums.is_null(2)); + + let state = accumulator.state(EmitTo::Block)?; + assert_eq!( + state[0].as_primitive::().values().as_ref(), + &[1, 1, 1] + ); + assert_eq!( + state[1].as_primitive::().values().as_ref(), + &[4.0, 5.0, 6.0] + ); + + let state = accumulator.state(EmitTo::Block)?; + assert_eq!( + state[0].as_primitive::().values().as_ref(), + &[1] + ); + assert_eq!( + state[1].as_primitive::().values().as_ref(), + &[7.0] + ); + + Ok(()) + } + + #[test] + fn blocked_avg_merge_evaluate_blocks() -> Result<()> { + let mut accumulator = + BlockedAvgGroupsAccumulator::new(AvgStateLayout::CountSumUInt64, 2); + let counts: ArrayRef = Arc::new(UInt64Array::from(vec![2, 1, 3, 4, 5])); + let sums: ArrayRef = + Arc::new(Float64Array::from(vec![10.0, 4.0, 9.0, 20.0, 15.0])); + let group_indices = vec![0, 1, 2, 3, 4]; + + accumulator.merge_batch(&[counts, sums], &group_indices, None, 5)?; + + let output = accumulator.evaluate(EmitTo::Block)?; + assert_eq!( + output.as_primitive::().values().as_ref(), + &[5.0, 4.0] + ); + + let output = accumulator.evaluate(EmitTo::Block)?; + assert_eq!( + output.as_primitive::().values().as_ref(), + &[3.0, 5.0] + ); + + let output = accumulator.evaluate(EmitTo::Block)?; + assert_eq!( + output.as_primitive::().values().as_ref(), + &[3.0] + ); + + Ok(()) + } + + #[test] + fn blocked_avg_spark_state_layout() -> Result<()> { + let mut accumulator = + BlockedAvgGroupsAccumulator::new(AvgStateLayout::SumCountInt64, 2); + let values: ArrayRef = Arc::new(Float64Array::from(vec![2.0, 4.0, 6.0])); + let group_indices = vec![0, 1, 2]; + + accumulator.update_batch(&[values], &group_indices, None, 3)?; + + let state = accumulator.state(EmitTo::Block)?; + assert_eq!( + state[0].as_primitive::().values().as_ref(), + &[2.0, 4.0] + ); + assert_eq!( + state[1].as_primitive::().values().as_ref(), + &[1, 1] + ); + + let state = accumulator.state(EmitTo::Block)?; + assert_eq!( + state[0].as_primitive::().values().as_ref(), + &[6.0] + ); + assert_eq!(state[1].as_primitive::().values().as_ref(), &[1]); + + Ok(()) + } +} From c719ff9a6a4ea8d24d82d2b5c77ca94fd20ec9a1 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Mon, 1 Jun 2026 15:57:01 +0800 Subject: [PATCH 4/8] show aggregate block state in explain --- .../src/aggregates/group_values/mod.rs | 7 +- .../physical-plan/src/aggregates/mod.rs | 64 ++++++++++++++++++- .../src/aggregates/raw_partial_hash.rs | 37 +++++++++-- 3 files changed, 99 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 3b3a0b6b90f0..1a296ccf9b38 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -151,8 +151,7 @@ pub(crate) fn new_unordered_blocked_group_values( schema: &SchemaRef, block_size: usize, ) -> Result>> { - if schema.fields.len() == 1 && matches!(schema.fields[0].data_type(), DataType::Int64) - { + if supports_blocked_group_values(schema) { return Ok(Some(Box::new(GroupValuesPrimitiveBlock::::new( schema.fields[0].data_type().clone(), block_size, @@ -162,6 +161,10 @@ pub(crate) fn new_unordered_blocked_group_values( Ok(None) } +pub(crate) fn supports_blocked_group_values(schema: &SchemaRef) -> bool { + schema.fields.len() == 1 && matches!(schema.fields[0].data_type(), DataType::Int64) +} + fn new_group_values_with_ordering( schema: SchemaRef, ordered: bool, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e3da94459db1..4553df59dc73 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -23,7 +23,10 @@ use std::sync::Arc; use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; use crate::aggregates::{ no_grouping::AggregateStream, - raw_partial_hash::{PartialFinalHashAggregateStream, RawPartialHashAggregateStream}, + raw_partial_hash::{ + PartialFinalHashAggregateStream, RawPartialHashAggregateStream, + can_use_blocked_hash_aggregate, + }, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; @@ -1054,6 +1057,12 @@ impl AggregateExec { "GroupedHashAggregateStream" } + fn uses_blocked_hash_stream_for_display(&self) -> bool { + (self.can_use_raw_partial_hash_stream() + || self.can_use_partial_final_hash_stream()) + && can_use_blocked_hash_aggregate(self).unwrap_or(false) + } + fn should_display_stream_name(&self, t: DisplayFormatType) -> bool { matches!(t, DisplayFormatType::Verbose) || self.can_use_raw_partial_hash_stream() @@ -1541,6 +1550,9 @@ impl DisplayAs for AggregateExec { if self.should_display_stream_name(t) { write!(f, ", stream={}", self.stream_name_for_display())?; + if self.uses_blocked_hash_stream_for_display() { + write!(f, ", blocked=true")?; + } } } DisplayFormatType::TreeRender => { @@ -3146,6 +3158,56 @@ mod tests { Ok(()) } + #[test] + fn grouped_avg_explain_shows_blocked_hash_stream() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("g", DataType::Int64, false), + Field::new("v", DataType::Float64, false), + ])); + let input = TestMemoryExec::try_new_exec(&[vec![]], Arc::clone(&schema), None)?; + let group_by = + PhysicalGroupBy::new_single(vec![(col("g", &schema)?, "g".to_string())]); + let aggregates: Vec> = vec![Arc::new( + AggregateExprBuilder::new(avg_udaf(), vec![col("v", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(v)") + .build()?, + )]; + + let partial_aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by.clone(), + aggregates.clone(), + vec![None], + input, + Arc::clone(&schema), + )?); + assert!( + crate::displayable(partial_aggregate.as_ref()) + .one_line() + .to_string() + .contains("stream=RawPartialHashAggregateStream, blocked=true") + ); + + let merge = Arc::new(CoalescePartitionsExec::new(partial_aggregate)); + let final_aggregate = AggregateExec::try_new( + AggregateMode::Final, + group_by.as_final(), + aggregates, + vec![None], + merge, + Arc::clone(&schema), + )?; + assert!( + crate::displayable(&final_aggregate) + .one_line() + .to_string() + .contains("stream=PartialFinalHashAggregateStream, blocked=true") + ); + + Ok(()) + } + #[tokio::test] async fn test_drop_cancel_without_groups() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs index d1bdb8d1f380..2131978550b0 100644 --- a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -52,7 +52,7 @@ use futures::stream::{Stream, StreamExt}; use super::group_values::{ GroupByMetrics, GroupValues, new_unordered_blocked_group_values, - new_unordered_group_values, + new_unordered_group_values, supports_blocked_group_values, }; use super::row_hash::create_group_accumulator; use super::{ @@ -434,10 +434,9 @@ impl GroupsAccumulator for BlockedAvgGroupsAccumulator { } } -fn create_blocked_group_accumulator( +fn blocked_avg_state_layout( agg_expr: &Arc, - block_size: usize, -) -> Result>> { +) -> Result> { if agg_expr.fun().name() != "avg" || agg_expr.is_distinct() { return Ok(None); } @@ -461,6 +460,14 @@ fn create_blocked_group_accumulator( _ => None, }; + Ok(state_layout) +} + +fn create_blocked_group_accumulator( + agg_expr: &Arc, + block_size: usize, +) -> Result>> { + let state_layout = blocked_avg_state_layout(agg_expr)?; Ok(state_layout.map(|state_layout| { Box::new(BlockedAvgGroupsAccumulator::new(state_layout, block_size)) as Box @@ -482,6 +489,25 @@ fn create_blocked_group_accumulators( Ok(Some(accumulators)) } +pub(crate) fn can_use_blocked_hash_aggregate(agg: &AggregateExec) -> Result { + if agg.group_by.has_grouping_set() || agg.group_by.groups().len() != 1 { + return Ok(false); + } + + let group_schema = agg.group_by.group_schema(&agg.input().schema())?; + if !supports_blocked_group_values(&group_schema) { + return Ok(false); + } + + for agg_expr in agg.aggr_expr.iter() { + if blocked_avg_state_layout(agg_expr)?.is_none() { + return Ok(false); + } + } + + Ok(true) +} + impl HashAggregateAccumulator { fn new( arguments: Vec>, @@ -649,8 +675,7 @@ impl AggregateHashTable { )?; let group_schema = agg.group_by.group_schema(&input_schema)?; - let can_use_blocked_output = - !agg.group_by.has_grouping_set() && agg.group_by.groups().len() == 1; + let can_use_blocked_output = can_use_blocked_hash_aggregate(agg)?; let blocked_group_values = if can_use_blocked_output { new_unordered_blocked_group_values(&group_schema, batch_size)? From f7b3847541206be7a22754f928343947354d757e Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Tue, 2 Jun 2026 10:25:35 +0800 Subject: [PATCH 5/8] hand optimize accumulator --- .../src/aggregates/raw_partial_hash.rs | 269 +++++++----------- 1 file changed, 101 insertions(+), 168 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs index 2131978550b0..52a75f0c97c6 100644 --- a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -34,10 +34,10 @@ use std::mem::size_of; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow::array::types::{Float64Type, Int64Type, UInt64Type}; +use arrow::array::types::{Float64Type, UInt64Type}; use arrow::array::{ - Array, ArrayRef, AsArray, BooleanArray, Float64Array, Int64Array, NullBufferBuilder, - UInt64Array, new_null_array, + Array, ArrayRef, AsArray, BooleanArray, Float64Array, NullBufferBuilder, UInt64Array, + new_null_array, }; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -46,6 +46,7 @@ use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use futures::ready; use futures::stream::{Stream, StreamExt}; @@ -101,33 +102,49 @@ struct EvaluatedAggregateBatch { accumulator_args: Vec, } +/// Avg accumulator for primitive types. +/// +/// It is mostly equivalent to `AvgGroupsAccumulator`, but uses a block-based +/// allocation policy for internal buffers: +/// +/// block_size: 100 +/// counts: [100 states], [100 states], [100 states], ... +/// sums : [100 states], [100 states], [100 states], ... +/// +/// The `group` index passed to `GroupsAccumulator` APIs is interpreted relative +/// to the accumulator's start. The block index and index within that block are +/// handled internally. +/// +/// For example, with `block_size` 100, group index 101 maps to block index +/// `101 / 100 == 1` and index within the block `101 % 100 == 1`. #[derive(Debug)] struct BlockedAvgGroupsAccumulator { - state_layout: AvgStateLayout, block_size: usize, counts: Vec>, sums: Vec>, + null_state: NullState, len: usize, } -#[derive(Debug, Clone, Copy)] -enum AvgStateLayout { - CountSumUInt64, - SumCountInt64, -} - impl BlockedAvgGroupsAccumulator { - fn new(state_layout: AvgStateLayout, block_size: usize) -> Self { + fn new(block_size: usize) -> Self { assert!(block_size > 0); Self { - state_layout, block_size, counts: vec![], sums: vec![], + null_state: NullState::new(), len: 0, } } + /// Ensure there is enough capacity by allocating enough blocks. + /// + /// For example: + /// 1. There are 2 blocks with size 100 + /// 2. Call `self.ensure_capacity(260)` + /// 3. A new block with the configured block size is allocated, so there is enough + /// room for 260 groups fn ensure_capacity(&mut self, total_num_groups: usize) { let required_blocks = total_num_groups.div_ceil(self.block_size); while self.counts.len() < required_blocks { @@ -139,22 +156,6 @@ impl BlockedAvgGroupsAccumulator { self.len = self.len.max(total_num_groups); } - fn add_value(&mut self, group_index: usize, value: f64) { - debug_assert!(group_index < self.len); - let block_idx = group_index / self.block_size; - let value_idx = group_index % self.block_size; - self.counts[block_idx][value_idx] += 1; - self.sums[block_idx][value_idx] += value; - } - - fn merge_value(&mut self, group_index: usize, count: u64, sum: f64) { - debug_assert!(group_index < self.len); - let block_idx = group_index / self.block_size; - let value_idx = group_index % self.block_size; - self.counts[block_idx][value_idx] += count; - self.sums[block_idx][value_idx] += sum; - } - fn values_range( blocks: &[Box<[T]>], block_size: usize, @@ -233,6 +234,7 @@ impl BlockedAvgGroupsAccumulator { } fn take_values(&mut self, emit_to: EmitTo) -> (Vec, Vec) { + self.null_state = NullState::new(); match emit_to { EmitTo::All => self.take_all_values(), EmitTo::Block => self.take_block_values(), @@ -257,10 +259,6 @@ impl BlockedAvgGroupsAccumulator { .map(|filter| filter.is_valid(row) && filter.value(row)) .unwrap_or(true) } - - fn count_state_i64(counts: &[u64]) -> Vec { - counts.iter().map(|count| *count as i64).collect() - } } impl GroupsAccumulator for BlockedAvgGroupsAccumulator { @@ -275,19 +273,24 @@ impl GroupsAccumulator for BlockedAvgGroupsAccumulator { let values = values[0].as_primitive::(); self.ensure_capacity(total_num_groups); - if values.null_count() == 0 && opt_filter.is_none() { - for (&group_index, &value) in group_indices.iter().zip(values.values().iter()) - { - self.add_value(group_index, value); - } - } else { - for (row, &group_index) in group_indices.iter().enumerate() { - if !Self::filter_is_valid(opt_filter, row) || values.is_null(row) { - continue; - } - self.add_value(group_index, values.value(row)); - } - } + let counts = &mut self.counts; + let sums = &mut self.sums; + let block_size = self.block_size; + let len = self.len; + + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + debug_assert!(group_index < len); + let block_idx = group_index / block_size; + let value_idx = group_index % block_size; + counts[block_idx][value_idx] += 1; + sums[block_idx][value_idx] += value; + }, + ); Ok(()) } @@ -314,19 +317,10 @@ impl GroupsAccumulator for BlockedAvgGroupsAccumulator { let (counts, sums) = self.take_values(emit_to); let nulls = Self::nulls_for_counts(&counts); - match self.state_layout { - AvgStateLayout::CountSumUInt64 => Ok(vec![ - Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, - Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, - ]), - AvgStateLayout::SumCountInt64 => Ok(vec![ - Arc::new(Float64Array::new(sums.into(), nulls.clone())) as ArrayRef, - Arc::new(Int64Array::new( - Self::count_state_i64(&counts).into(), - nulls, - )) as ArrayRef, - ]), - } + Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, + Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, + ]) } fn merge_batch( @@ -339,46 +333,38 @@ impl GroupsAccumulator for BlockedAvgGroupsAccumulator { assert_eq!(values.len(), 2, "two arguments to merge_batch"); self.ensure_capacity(total_num_groups); - match self.state_layout { - AvgStateLayout::CountSumUInt64 => { - let partial_counts = values[0].as_primitive::(); - let partial_sums = values[1].as_primitive::(); - - for (row, &group_index) in group_indices.iter().enumerate() { - if !Self::filter_is_valid(opt_filter, row) - || partial_counts.is_null(row) - || partial_sums.is_null(row) - { - continue; - } + let partial_counts = values[0].as_primitive::(); + let partial_sums = values[1].as_primitive::(); - self.merge_value( - group_index, - partial_counts.value(row), - partial_sums.value(row), - ); - } - } - AvgStateLayout::SumCountInt64 => { - let partial_sums = values[0].as_primitive::(); - let partial_counts = values[1].as_primitive::(); - - for (row, &group_index) in group_indices.iter().enumerate() { - if !Self::filter_is_valid(opt_filter, row) - || partial_counts.is_null(row) - || partial_sums.is_null(row) - { - continue; - } + let counts = &mut self.counts; + let block_size = self.block_size; + let len = self.len; + self.null_state.accumulate( + group_indices, + partial_counts, + opt_filter, + total_num_groups, + |group_index, partial_count| { + debug_assert!(group_index < len); + let block_idx = group_index / block_size; + let value_idx = group_index % block_size; + counts[block_idx][value_idx] += partial_count; + }, + ); - self.merge_value( - group_index, - partial_counts.value(row) as u64, - partial_sums.value(row), - ); - } - } - } + let sums = &mut self.sums; + self.null_state.accumulate( + group_indices, + partial_sums, + opt_filter, + total_num_groups, + |group_index, partial_sum| { + debug_assert!(group_index < len); + let block_idx = group_index / block_size; + let value_idx = group_index % block_size; + sums[block_idx][value_idx] += partial_sum; + }, + ); Ok(()) } @@ -407,19 +393,10 @@ impl GroupsAccumulator for BlockedAvgGroupsAccumulator { } let nulls = nulls.finish(); - match self.state_layout { - AvgStateLayout::CountSumUInt64 => Ok(vec![ - Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, - Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, - ]), - AvgStateLayout::SumCountInt64 => Ok(vec![ - Arc::new(Float64Array::new(sums.into(), nulls.clone())) as ArrayRef, - Arc::new(Int64Array::new( - Self::count_state_i64(&counts).into(), - nulls, - )) as ArrayRef, - ]), - } + Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, + Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, + ]) } fn supports_convert_to_state(&self) -> bool { @@ -431,47 +408,34 @@ impl GroupsAccumulator for BlockedAvgGroupsAccumulator { + self.sums.len() * self.block_size * size_of::() + self.counts.allocated_size() + self.sums.allocated_size() + + self.null_state.size() } } -fn blocked_avg_state_layout( - agg_expr: &Arc, -) -> Result> { +fn supports_blocked_avg_state(agg_expr: &Arc) -> Result { if agg_expr.fun().name() != "avg" || agg_expr.is_distinct() { - return Ok(None); + return Ok(false); } let state_fields = agg_expr.state_fields()?; - let state_layout = match state_fields.as_slice() { + Ok(matches!( + state_fields.as_slice(), [count, sum] if matches!(count.data_type(), DataType::UInt64) && matches!(sum.data_type(), DataType::Float64) - && matches!(agg_expr.field().data_type(), DataType::Float64) => - { - Some(AvgStateLayout::CountSumUInt64) - } - [sum, count] - if matches!(sum.data_type(), DataType::Float64) - && matches!(count.data_type(), DataType::Int64) - && matches!(agg_expr.field().data_type(), DataType::Float64) => - { - Some(AvgStateLayout::SumCountInt64) - } - _ => None, - }; - - Ok(state_layout) + && matches!(agg_expr.field().data_type(), DataType::Float64) + )) } fn create_blocked_group_accumulator( agg_expr: &Arc, block_size: usize, ) -> Result>> { - let state_layout = blocked_avg_state_layout(agg_expr)?; - Ok(state_layout.map(|state_layout| { - Box::new(BlockedAvgGroupsAccumulator::new(state_layout, block_size)) - as Box - })) + if supports_blocked_avg_state(agg_expr)? { + Ok(Some(Box::new(BlockedAvgGroupsAccumulator::new(block_size)))) + } else { + Ok(None) + } } fn create_blocked_group_accumulators( @@ -500,7 +464,7 @@ pub(crate) fn can_use_blocked_hash_aggregate(agg: &AggregateExec) -> Result Result<()> { - let mut accumulator = - BlockedAvgGroupsAccumulator::new(AvgStateLayout::CountSumUInt64, 3); + let mut accumulator = BlockedAvgGroupsAccumulator::new(3); let values: ArrayRef = Arc::new(Float64Array::from(vec![ Some(1.0), Some(2.0), @@ -1483,8 +1446,7 @@ mod tests { #[test] fn blocked_avg_merge_evaluate_blocks() -> Result<()> { - let mut accumulator = - BlockedAvgGroupsAccumulator::new(AvgStateLayout::CountSumUInt64, 2); + let mut accumulator = BlockedAvgGroupsAccumulator::new(2); let counts: ArrayRef = Arc::new(UInt64Array::from(vec![2, 1, 3, 4, 5])); let sums: ArrayRef = Arc::new(Float64Array::from(vec![10.0, 4.0, 9.0, 20.0, 15.0])); @@ -1512,33 +1474,4 @@ mod tests { Ok(()) } - - #[test] - fn blocked_avg_spark_state_layout() -> Result<()> { - let mut accumulator = - BlockedAvgGroupsAccumulator::new(AvgStateLayout::SumCountInt64, 2); - let values: ArrayRef = Arc::new(Float64Array::from(vec![2.0, 4.0, 6.0])); - let group_indices = vec![0, 1, 2]; - - accumulator.update_batch(&[values], &group_indices, None, 3)?; - - let state = accumulator.state(EmitTo::Block)?; - assert_eq!( - state[0].as_primitive::().values().as_ref(), - &[2.0, 4.0] - ); - assert_eq!( - state[1].as_primitive::().values().as_ref(), - &[1, 1] - ); - - let state = accumulator.state(EmitTo::Block)?; - assert_eq!( - state[0].as_primitive::().values().as_ref(), - &[6.0] - ); - assert_eq!(state[1].as_primitive::().values().as_ref(), &[1]); - - Ok(()) - } } From f97a3bdf4460e79f0b78d0311f86d598cd992141 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Tue, 2 Jun 2026 11:42:13 +0800 Subject: [PATCH 6/8] cp --- .../expr-common/src/groups_accumulator.rs | 20 +- datafusion/functions-aggregate/src/average.rs | 391 +++++++++++++- .../src/aggregates/group_values/mod.rs | 67 ++- .../group_values/multi_group_by/mod.rs | 9 +- .../src/aggregates/group_values/row.rs | 9 +- .../single_group_by/blocked_primitive.rs | 4 + .../group_values/single_group_by/boolean.rs | 9 +- .../group_values/single_group_by/bytes.rs | 9 +- .../single_group_by/bytes_view.rs | 10 +- .../group_values/single_group_by/primitive.rs | 9 +- .../src/aggregates/raw_partial_hash.rs | 497 +++--------------- .../physical-plan/src/aggregates/row_hash.rs | 41 ++ 12 files changed, 615 insertions(+), 460 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index ad67f763302b..d5635ace4418 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -29,7 +29,7 @@ pub enum EmitTo { /// /// This is intended for accumulators and group-value stores whose internal /// representation is already split into fixed-size blocks. Implementations - /// without internal blocking may treat this the same as [`Self::All`]. + /// that do not advertise blocked emit support may return an internal error. /// /// Callers should only use this once no further updates will arrive for the /// current groups. @@ -256,6 +256,24 @@ pub trait GroupsAccumulator: Send + std::any::Any { false } + /// Creates an accumulator that stores and emits state in bounded blocks. + /// + /// This is used by hash aggregation paths that pair a blocked group-value + /// store with blocked accumulator state so output batches can be produced + /// without first materializing all groups contiguously. + fn create_blocked_accumulator( + &self, + _block_size: usize, + ) -> Result>> { + Ok(None) + } + + /// Returns true when [`Self::state`] and [`Self::evaluate`] support + /// [`EmitTo::Block`] by emitting one bounded block of accumulator state. + fn supports_blocked_emit(&self) -> bool { + false + } + /// Amount of memory used to store the state of this accumulator, /// in bytes. /// diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index ddeb9b0870a1..cffa0f528ca9 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -19,7 +19,8 @@ use arrow::array::{ Array, ArrayRef, ArrowNativeTypeOp, ArrowNumericType, ArrowPrimitiveType, AsArray, - BooleanArray, PrimitiveArray, PrimitiveBuilder, UInt64Array, + BooleanArray, Float64Array, NullBufferBuilder, PrimitiveArray, PrimitiveBuilder, + UInt64Array, }; use arrow::compute::sum; @@ -33,6 +34,7 @@ use arrow::datatypes::{ }; use datafusion_common::types::{NativeType, logical_float64}; use datafusion_common::{Result, ScalarValue, exec_err, not_impl_err}; +use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ @@ -790,6 +792,138 @@ where avg_fn: F, } +/// Avg accumulator that stores its per-group state in fixed-size blocks. +#[derive(Debug)] +struct BlockedAvgGroupsAccumulator { + block_size: usize, + counts: Vec>, + sums: Vec>, + null_state: NullState, + len: usize, +} + +impl BlockedAvgGroupsAccumulator { + fn new(block_size: usize) -> Self { + assert!(block_size > 0); + Self { + block_size, + counts: vec![], + sums: vec![], + null_state: NullState::new(), + len: 0, + } + } + + fn ensure_capacity(&mut self, total_num_groups: usize) { + let required_blocks = total_num_groups.div_ceil(self.block_size); + while self.counts.len() < required_blocks { + self.counts + .push(vec![0; self.block_size].into_boxed_slice()); + self.sums + .push(vec![0.0; self.block_size].into_boxed_slice()); + } + self.len = self.len.max(total_num_groups); + } + + fn values_range( + blocks: &[Box<[T]>], + block_size: usize, + start: usize, + len: usize, + ) -> Vec { + let mut output = Vec::with_capacity(len); + let mut remaining = len; + let mut group_id = start; + + while remaining > 0 { + let block_idx = group_id / block_size; + let offset = group_id % block_size; + let take = remaining.min(block_size - offset); + output.extend_from_slice(&blocks[block_idx][offset..offset + take]); + remaining -= take; + group_id += take; + } + + output + } + + fn rebuild_from_values(&mut self, counts: &[u64], sums: &[f64]) { + debug_assert_eq!(counts.len(), sums.len()); + self.counts.clear(); + self.sums.clear(); + self.len = counts.len(); + + for chunk in counts.chunks(self.block_size) { + let mut block = vec![0; self.block_size].into_boxed_slice(); + block[..chunk.len()].copy_from_slice(chunk); + self.counts.push(block); + } + + for chunk in sums.chunks(self.block_size) { + let mut block = vec![0.0; self.block_size].into_boxed_slice(); + block[..chunk.len()].copy_from_slice(chunk); + self.sums.push(block); + } + } + + fn take_all_values(&mut self) -> (Vec, Vec) { + let counts = Self::values_range(&self.counts, self.block_size, 0, self.len); + let sums = Self::values_range(&self.sums, self.block_size, 0, self.len); + self.counts.clear(); + self.sums.clear(); + self.len = 0; + (counts, sums) + } + + fn take_block_values(&mut self) -> (Vec, Vec) { + let emit_len = self.len.min(self.block_size); + + let mut counts = self.counts.remove(0).into_vec(); + counts.truncate(emit_len); + + let mut sums = self.sums.remove(0).into_vec(); + sums.truncate(emit_len); + + self.len -= emit_len; + (counts, sums) + } + + fn take_first_values(&mut self, n: usize) -> (Vec, Vec) { + let n = n.min(self.len); + let counts = Self::values_range(&self.counts, self.block_size, 0, n); + let sums = Self::values_range(&self.sums, self.block_size, 0, n); + + let remaining_counts = + Self::values_range(&self.counts, self.block_size, n, self.len - n); + let remaining_sums = + Self::values_range(&self.sums, self.block_size, n, self.len - n); + self.rebuild_from_values(&remaining_counts, &remaining_sums); + + (counts, sums) + } + + fn take_values(&mut self, emit_to: EmitTo) -> (Vec, Vec) { + self.null_state = NullState::new(); + match emit_to { + EmitTo::All => self.take_all_values(), + EmitTo::Block => self.take_block_values(), + EmitTo::First(n) => self.take_first_values(n), + } + } + + fn nulls_for_counts(counts: &[u64]) -> Option { + let mut nulls = NullBufferBuilder::new(counts.len()); + for count in counts { + if *count == 0 { + nulls.append_null(); + } else { + nulls.append_non_null(); + } + } + nulls.finish() + } +} + impl AvgGroupsAccumulator where T: ArrowNumericType + Send, @@ -970,7 +1104,262 @@ where true } + fn create_blocked_accumulator( + &self, + block_size: usize, + ) -> Result>> { + if self.sum_data_type == DataType::Float64 + && self.return_data_type == DataType::Float64 + { + Ok(Some(Box::new(BlockedAvgGroupsAccumulator::new(block_size)))) + } else { + Ok(None) + } + } + fn size(&self) -> usize { self.counts.capacity() * size_of::() + self.sums.capacity() * size_of::() } } + +impl GroupsAccumulator for BlockedAvgGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + self.ensure_capacity(total_num_groups); + + let counts = &mut self.counts; + let sums = &mut self.sums; + let block_size = self.block_size; + let len = self.len; + + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + debug_assert!(group_index < len); + let block_idx = group_index / block_size; + let value_idx = group_index % block_size; + counts[block_idx][value_idx] += 1; + sums[block_idx][value_idx] += value; + }, + ); + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let (counts, sums) = self.take_values(emit_to); + let mut values = Vec::with_capacity(counts.len()); + let mut nulls = NullBufferBuilder::new(counts.len()); + + for (count, sum) in counts.into_iter().zip(sums) { + if count == 0 { + values.push(0.0); + nulls.append_null(); + } else { + values.push(sum / count as f64); + nulls.append_non_null(); + } + } + + Ok(Arc::new(Float64Array::new(values.into(), nulls.finish()))) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + let (counts, sums) = self.take_values(emit_to); + let nulls = Self::nulls_for_counts(&counts); + + Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, + Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, + ]) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to merge_batch"); + self.ensure_capacity(total_num_groups); + + let partial_counts = values[0].as_primitive::(); + let partial_sums = values[1].as_primitive::(); + + let counts = &mut self.counts; + let block_size = self.block_size; + let len = self.len; + self.null_state.accumulate( + group_indices, + partial_counts, + opt_filter, + total_num_groups, + |group_index, partial_count| { + debug_assert!(group_index < len); + let block_idx = group_index / block_size; + let value_idx = group_index % block_size; + counts[block_idx][value_idx] += partial_count; + }, + ); + + let sums = &mut self.sums; + self.null_state.accumulate( + group_indices, + partial_sums, + opt_filter, + total_num_groups, + |group_index, partial_sum| { + debug_assert!(group_index < len); + let block_idx = group_index / block_size; + let value_idx = group_index % block_size; + sums[block_idx][value_idx] += partial_sum; + }, + ); + + Ok(()) + } + + fn convert_to_state( + &self, + values: &[ArrayRef], + opt_filter: Option<&BooleanArray>, + ) -> Result> { + assert_eq!(values.len(), 1, "single argument to convert_to_state"); + let values = values[0].as_primitive::(); + let mut counts = Vec::with_capacity(values.len()); + let mut sums = Vec::with_capacity(values.len()); + let mut nulls = NullBufferBuilder::new(values.len()); + + for row in 0..values.len() { + if opt_filter + .map(|filter| filter.is_valid(row) && filter.value(row)) + .unwrap_or(true) + && values.is_valid(row) + { + counts.push(1); + sums.push(values.value(row)); + nulls.append_non_null(); + } else { + counts.push(0); + sums.push(0.0); + nulls.append_null(); + } + } + + let nulls = nulls.finish(); + Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, + Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, + ]) + } + + fn supports_convert_to_state(&self) -> bool { + true + } + + fn supports_blocked_emit(&self) -> bool { + true + } + + fn size(&self) -> usize { + self.counts.len() * self.block_size * size_of::() + + self.sums.len() * self.block_size * size_of::() + + self.counts.allocated_size() + + self.sums.allocated_size() + + self.null_state.size() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn blocked_avg_state_emit_blocks() -> Result<()> { + let mut accumulator = BlockedAvgGroupsAccumulator::new(3); + let values: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(1.0), + Some(2.0), + None, + Some(4.0), + Some(5.0), + Some(6.0), + Some(7.0), + ])); + let group_indices = vec![0, 1, 2, 3, 4, 5, 6]; + + accumulator.update_batch(&[values], &group_indices, None, 7)?; + + let state = accumulator.state(EmitTo::Block)?; + let counts = state[0].as_primitive::(); + let sums = state[1].as_primitive::(); + assert_eq!(counts.values().as_ref(), &[1, 1, 0]); + assert_eq!(sums.values().as_ref(), &[1.0, 2.0, 0.0]); + assert!(counts.is_null(2)); + assert!(sums.is_null(2)); + + let state = accumulator.state(EmitTo::Block)?; + assert_eq!( + state[0].as_primitive::().values().as_ref(), + &[1, 1, 1] + ); + assert_eq!( + state[1].as_primitive::().values().as_ref(), + &[4.0, 5.0, 6.0] + ); + + let state = accumulator.state(EmitTo::Block)?; + assert_eq!( + state[0].as_primitive::().values().as_ref(), + &[1] + ); + assert_eq!( + state[1].as_primitive::().values().as_ref(), + &[7.0] + ); + + Ok(()) + } + + #[test] + fn blocked_avg_merge_evaluate_blocks() -> Result<()> { + let mut accumulator = BlockedAvgGroupsAccumulator::new(2); + let counts: ArrayRef = Arc::new(UInt64Array::from(vec![2, 1, 3, 4, 5])); + let sums: ArrayRef = + Arc::new(Float64Array::from(vec![10.0, 4.0, 9.0, 20.0, 15.0])); + let group_indices = vec![0, 1, 2, 3, 4]; + + accumulator.merge_batch(&[counts, sums], &group_indices, None, 5)?; + + let output = accumulator.evaluate(EmitTo::Block)?; + assert_eq!( + output.as_primitive::().values().as_ref(), + &[5.0, 4.0] + ); + + let output = accumulator.evaluate(EmitTo::Block)?; + assert_eq!( + output.as_primitive::().values().as_ref(), + &[3.0, 5.0] + ); + + let output = accumulator.evaluate(EmitTo::Block)?; + assert_eq!( + output.as_primitive::().values().as_ref(), + &[3.0] + ); + + Ok(()) + } +} diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 1a296ccf9b38..007adc6bc11d 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -18,10 +18,9 @@ //! [`GroupValues`] trait for storing and interning group keys use arrow::array::types::{ - Date32Type, Date64Type, Decimal128Type, Int64Type, Time32MillisecondType, - Time32SecondType, Time64MicrosecondType, Time64NanosecondType, - TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, - TimestampSecondType, + Date32Type, Date64Type, Decimal128Type, Time32MillisecondType, Time32SecondType, + Time64MicrosecondType, Time64NanosecondType, TimestampMicrosecondType, + TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }; use arrow::array::{ArrayRef, downcast_primitive}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; @@ -113,6 +112,12 @@ pub trait GroupValues: Send { /// Emits the group values fn emit(&mut self, emit_to: EmitTo) -> Result>; + /// Returns true when [`Self::emit`] supports [`EmitTo::Block`] by emitting + /// one bounded block of group values. + fn supports_blocked_emit(&self) -> bool { + false + } + /// Clear the contents and shrink the capacity to the size of the batch (free up memory usage) fn clear_shrink(&mut self, num_rows: usize); } @@ -151,20 +156,56 @@ pub(crate) fn new_unordered_blocked_group_values( schema: &SchemaRef, block_size: usize, ) -> Result>> { - if supports_blocked_group_values(schema) { - return Ok(Some(Box::new(GroupValuesPrimitiveBlock::::new( - schema.fields[0].data_type().clone(), - block_size, - )))); + if schema.fields.len() == 1 { + let d = schema.fields[0].data_type(); + + macro_rules! downcast_helper { + ($t:ty, $d:ident) => { + return Ok(Some(Box::new(GroupValuesPrimitiveBlock::<$t>::new( + $d.clone(), + block_size, + )))) + }; + } + + downcast_primitive! { + d => (downcast_helper, d), + _ => {} + } + + match d { + DataType::Date32 => { + downcast_helper!(Date32Type, d); + } + DataType::Date64 => { + downcast_helper!(Date64Type, d); + } + DataType::Time32(t) => match t { + TimeUnit::Second => downcast_helper!(Time32SecondType, d), + TimeUnit::Millisecond => downcast_helper!(Time32MillisecondType, d), + _ => {} + }, + DataType::Time64(t) => match t { + TimeUnit::Microsecond => downcast_helper!(Time64MicrosecondType, d), + TimeUnit::Nanosecond => downcast_helper!(Time64NanosecondType, d), + _ => {} + }, + DataType::Timestamp(t, _tz) => match t { + TimeUnit::Second => downcast_helper!(TimestampSecondType, d), + TimeUnit::Millisecond => downcast_helper!(TimestampMillisecondType, d), + TimeUnit::Microsecond => downcast_helper!(TimestampMicrosecondType, d), + TimeUnit::Nanosecond => downcast_helper!(TimestampNanosecondType, d), + }, + DataType::Decimal128(_, _) => { + downcast_helper!(Decimal128Type, d); + } + _ => {} + } } Ok(None) } -pub(crate) fn supports_blocked_group_values(schema: &SchemaRef) -> bool { - schema.fields.len() == 1 && matches!(schema.fields[0].data_type(), DataType::Int64) -} - fn new_group_values_with_ordering( schema: SchemaRef, ordered: bool, diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs index 876cea13f008..d7e98511268e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs @@ -41,7 +41,7 @@ use arrow::datatypes::{ }; use datafusion_common::hash_utils::RandomState; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::{Result, internal_datafusion_err, not_impl_err}; +use datafusion_common::{Result, internal_datafusion_err, internal_err, not_impl_err}; use datafusion_execution::memory_pool::proxy::{HashTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; @@ -1090,7 +1090,7 @@ impl GroupValues for GroupValuesColumn { fn emit(&mut self, emit_to: EmitTo) -> Result> { let mut output = match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { let group_values = mem::take(&mut self.group_values); debug_assert!(self.group_values.is_empty()); @@ -1099,6 +1099,11 @@ impl GroupValues for GroupValuesColumn { .map(|v| v.build()) .collect::>() } + EmitTo::Block => { + return internal_err!( + "EmitTo::Block is not supported by GroupValuesColumn" + ); + } EmitTo::First(n) => { let output = self .group_values diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index d45a0670d14c..6cd3283a2612 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -23,9 +23,9 @@ use arrow::array::{ use arrow::compute::cast; use arrow::datatypes::{DataType, SchemaRef}; use arrow::row::{RowConverter, Rows, SortField}; -use datafusion_common::Result; use datafusion_common::hash_utils::RandomState; use datafusion_common::hash_utils::create_hashes; +use datafusion_common::{Result, internal_err}; use datafusion_execution::memory_pool::proxy::{HashTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use hashbrown::hash_table::HashTable; @@ -196,18 +196,23 @@ impl GroupValues for GroupValuesRows { } fn emit(&mut self, emit_to: EmitTo) -> Result> { + if matches!(emit_to, EmitTo::Block) { + return internal_err!("EmitTo::Block is not supported by GroupValuesRows"); + } + let mut group_values = self .group_values .take() .expect("Can not emit from empty rows"); let mut output = match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { let output = self.row_converter.convert_rows(&group_values)?; group_values.clear(); self.map.clear(); output } + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) => { let groups_rows = group_values.iter().take(n); let output = self.row_converter.convert_rows(groups_rows)?; diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs index 55d7fab7b940..a0f4571f1772 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/blocked_primitive.rs @@ -303,6 +303,10 @@ where Ok(vec![output]) } + fn supports_blocked_emit(&self) -> bool { + true + } + fn clear_shrink(&mut self, num_rows: usize) { self.blocks.clear(); self.blocks.shrink_to(num_rows.div_ceil(self.block_size)); diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs index 2d599f9c394c..e2fe486ed473 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs @@ -20,7 +20,7 @@ use crate::aggregates::group_values::GroupValues; use arrow::array::{ ArrayRef, AsArray as _, BooleanArray, BooleanBufferBuilder, NullBufferBuilder, }; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_expr::EmitTo; use std::{mem::size_of, sync::Arc}; @@ -101,7 +101,12 @@ impl GroupValues for GroupValuesBoolean { let len = self.len(); let mut builder = BooleanBufferBuilder::new(len); let emit_count = match emit_to { - EmitTo::All | EmitTo::Block => len, + EmitTo::All => len, + EmitTo::Block => { + return internal_err!( + "EmitTo::Block is not supported by GroupValuesBoolean" + ); + } EmitTo::First(n) => n, }; builder.append_n(emit_count, false); diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs index 9ee6444855e4..7646d2cc0ad3 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs @@ -20,7 +20,7 @@ use std::mem::size_of; use crate::aggregates::group_values::GroupValues; use arrow::array::{Array, ArrayRef, OffsetSizeTrait}; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_expr::EmitTo; use datafusion_physical_expr_common::binary_map::{ArrowBytesMap, OutputType}; @@ -85,14 +85,19 @@ impl GroupValues for GroupValuesBytes { } fn emit(&mut self, emit_to: EmitTo) -> Result> { + if matches!(emit_to, EmitTo::Block) { + return internal_err!("EmitTo::Block is not supported by GroupValuesBytes"); + } + // Reset the map to default, and convert it into a single array let map_contents = self.map.take().into_state(); let group_values = match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { self.num_groups -= map_contents.len(); map_contents } + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) if n == self.len() => { self.num_groups -= map_contents.len(); map_contents diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs index ff43977ca540..65cde76e00b4 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs @@ -17,6 +17,7 @@ use crate::aggregates::group_values::GroupValues; use arrow::array::{Array, ArrayRef}; +use datafusion_common::internal_err; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewMap; @@ -87,14 +88,21 @@ impl GroupValues for GroupValuesBytesView { } fn emit(&mut self, emit_to: EmitTo) -> datafusion_common::Result> { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by GroupValuesBytesView" + ); + } + // Reset the map to default, and convert it into a single array let map_contents = self.map.take().into_state(); let group_values = match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { self.num_groups -= map_contents.len(); map_contents } + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) if n == self.len() => { self.num_groups -= map_contents.len(); map_contents diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs index 0e7b77e2c639..dc55b3606208 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs @@ -22,8 +22,8 @@ use arrow::array::{ cast::AsArray, }; use arrow::datatypes::{DataType, i256}; -use datafusion_common::Result; use datafusion_common::hash_utils::RandomState; +use datafusion_common::{Result, internal_err}; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; use half::f16; @@ -181,10 +181,15 @@ where } let array: PrimitiveArray = match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { self.map.clear(); build_primitive(std::mem::take(&mut self.values), self.null_group.take()) } + EmitTo::Block => { + return internal_err!( + "EmitTo::Block is not supported by GroupValuesPrimitive" + ); + } EmitTo::First(n) => { self.map.retain(|entry| { // Decrement group index by n diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs index 52a75f0c97c6..d8f1a8c288aa 100644 --- a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -30,32 +30,28 @@ use std::collections::HashMap; use std::marker::PhantomData; -use std::mem::size_of; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow::array::types::{Float64Type, UInt64Type}; -use arrow::array::{ - Array, ArrayRef, AsArray, BooleanArray, Float64Array, NullBufferBuilder, UInt64Array, - new_null_array, -}; -use arrow::datatypes::{DataType, SchemaRef}; +use arrow::array::{ArrayRef, AsArray, BooleanArray, new_null_array}; +use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_expr::{EmitTo, GroupsAccumulator}; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; -use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use futures::ready; use futures::stream::{Stream, StreamExt}; use super::group_values::{ GroupByMetrics, GroupValues, new_unordered_blocked_group_values, - new_unordered_group_values, supports_blocked_group_values, + new_unordered_group_values, +}; +use super::row_hash::{ + create_blocked_group_accumulator, create_blocked_group_accumulators, + create_group_accumulator, }; -use super::row_hash::create_group_accumulator; use super::{ AggregateExec, PhysicalGroupBy, aggregate_expressions, evaluate_group_by, group_id_array, max_duplicate_ordinal, @@ -102,369 +98,30 @@ struct EvaluatedAggregateBatch { accumulator_args: Vec, } -/// Avg accumulator for primitive types. -/// -/// It is mostly equivalent to `AvgGroupsAccumulator`, but uses a block-based -/// allocation policy for internal buffers: -/// -/// block_size: 100 -/// counts: [100 states], [100 states], [100 states], ... -/// sums : [100 states], [100 states], [100 states], ... -/// -/// The `group` index passed to `GroupsAccumulator` APIs is interpreted relative -/// to the accumulator's start. The block index and index within that block are -/// handled internally. -/// -/// For example, with `block_size` 100, group index 101 maps to block index -/// `101 / 100 == 1` and index within the block `101 % 100 == 1`. -#[derive(Debug)] -struct BlockedAvgGroupsAccumulator { - block_size: usize, - counts: Vec>, - sums: Vec>, - null_state: NullState, - len: usize, -} - -impl BlockedAvgGroupsAccumulator { - fn new(block_size: usize) -> Self { - assert!(block_size > 0); - Self { - block_size, - counts: vec![], - sums: vec![], - null_state: NullState::new(), - len: 0, - } - } - - /// Ensure there is enough capacity by allocating enough blocks. - /// - /// For example: - /// 1. There are 2 blocks with size 100 - /// 2. Call `self.ensure_capacity(260)` - /// 3. A new block with the configured block size is allocated, so there is enough - /// room for 260 groups - fn ensure_capacity(&mut self, total_num_groups: usize) { - let required_blocks = total_num_groups.div_ceil(self.block_size); - while self.counts.len() < required_blocks { - self.counts - .push(vec![0; self.block_size].into_boxed_slice()); - self.sums - .push(vec![0.0; self.block_size].into_boxed_slice()); - } - self.len = self.len.max(total_num_groups); - } - - fn values_range( - blocks: &[Box<[T]>], - block_size: usize, - start: usize, - len: usize, - ) -> Vec { - let mut output = Vec::with_capacity(len); - let mut remaining = len; - let mut group_id = start; - - while remaining > 0 { - let block_idx = group_id / block_size; - let offset = group_id % block_size; - let take = remaining.min(block_size - offset); - output.extend_from_slice(&blocks[block_idx][offset..offset + take]); - remaining -= take; - group_id += take; - } - - output - } - - fn rebuild_from_values(&mut self, counts: &[u64], sums: &[f64]) { - debug_assert_eq!(counts.len(), sums.len()); - self.counts.clear(); - self.sums.clear(); - self.len = counts.len(); - - for chunk in counts.chunks(self.block_size) { - let mut block = vec![0; self.block_size].into_boxed_slice(); - block[..chunk.len()].copy_from_slice(chunk); - self.counts.push(block); - } - - for chunk in sums.chunks(self.block_size) { - let mut block = vec![0.0; self.block_size].into_boxed_slice(); - block[..chunk.len()].copy_from_slice(chunk); - self.sums.push(block); - } - } - - fn take_all_values(&mut self) -> (Vec, Vec) { - let counts = Self::values_range(&self.counts, self.block_size, 0, self.len); - let sums = Self::values_range(&self.sums, self.block_size, 0, self.len); - self.counts.clear(); - self.sums.clear(); - self.len = 0; - (counts, sums) - } - - fn take_block_values(&mut self) -> (Vec, Vec) { - let emit_len = self.len.min(self.block_size); - - let mut counts = self.counts.remove(0).into_vec(); - counts.truncate(emit_len); - - let mut sums = self.sums.remove(0).into_vec(); - sums.truncate(emit_len); - - self.len -= emit_len; - (counts, sums) - } - - fn take_first_values(&mut self, n: usize) -> (Vec, Vec) { - let n = n.min(self.len); - let counts = Self::values_range(&self.counts, self.block_size, 0, n); - let sums = Self::values_range(&self.sums, self.block_size, 0, n); - - let remaining_counts = - Self::values_range(&self.counts, self.block_size, n, self.len - n); - let remaining_sums = - Self::values_range(&self.sums, self.block_size, n, self.len - n); - self.rebuild_from_values(&remaining_counts, &remaining_sums); - - (counts, sums) - } - - fn take_values(&mut self, emit_to: EmitTo) -> (Vec, Vec) { - self.null_state = NullState::new(); - match emit_to { - EmitTo::All => self.take_all_values(), - EmitTo::Block => self.take_block_values(), - EmitTo::First(n) => self.take_first_values(n), - } - } - - fn nulls_for_counts(counts: &[u64]) -> Option { - let mut nulls = NullBufferBuilder::new(counts.len()); - for count in counts { - if *count == 0 { - nulls.append_null(); - } else { - nulls.append_non_null(); - } - } - nulls.finish() - } - - fn filter_is_valid(opt_filter: Option<&BooleanArray>, row: usize) -> bool { - opt_filter - .map(|filter| filter.is_valid(row) && filter.value(row)) - .unwrap_or(true) - } +fn can_try_blocked_hash_aggregate(agg: &AggregateExec) -> bool { + !agg.group_by.has_grouping_set() && agg.group_by.groups().len() == 1 } -impl GroupsAccumulator for BlockedAvgGroupsAccumulator { - fn update_batch( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - assert_eq!(values.len(), 1, "single argument to update_batch"); - let values = values[0].as_primitive::(); - self.ensure_capacity(total_num_groups); - - let counts = &mut self.counts; - let sums = &mut self.sums; - let block_size = self.block_size; - let len = self.len; - - self.null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - debug_assert!(group_index < len); - let block_idx = group_index / block_size; - let value_idx = group_index % block_size; - counts[block_idx][value_idx] += 1; - sums[block_idx][value_idx] += value; - }, - ); - - Ok(()) - } - - fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let (counts, sums) = self.take_values(emit_to); - let mut values = Vec::with_capacity(counts.len()); - let mut nulls = NullBufferBuilder::new(counts.len()); - - for (count, sum) in counts.into_iter().zip(sums) { - if count == 0 { - values.push(0.0); - nulls.append_null(); - } else { - values.push(sum / count as f64); - nulls.append_non_null(); - } - } - - Ok(Arc::new(Float64Array::new(values.into(), nulls.finish()))) - } - - fn state(&mut self, emit_to: EmitTo) -> Result> { - let (counts, sums) = self.take_values(emit_to); - let nulls = Self::nulls_for_counts(&counts); - - Ok(vec![ - Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, - Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, - ]) - } - - fn merge_batch( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - assert_eq!(values.len(), 2, "two arguments to merge_batch"); - self.ensure_capacity(total_num_groups); - - let partial_counts = values[0].as_primitive::(); - let partial_sums = values[1].as_primitive::(); - - let counts = &mut self.counts; - let block_size = self.block_size; - let len = self.len; - self.null_state.accumulate( - group_indices, - partial_counts, - opt_filter, - total_num_groups, - |group_index, partial_count| { - debug_assert!(group_index < len); - let block_idx = group_index / block_size; - let value_idx = group_index % block_size; - counts[block_idx][value_idx] += partial_count; - }, - ); - - let sums = &mut self.sums; - self.null_state.accumulate( - group_indices, - partial_sums, - opt_filter, - total_num_groups, - |group_index, partial_sum| { - debug_assert!(group_index < len); - let block_idx = group_index / block_size; - let value_idx = group_index % block_size; - sums[block_idx][value_idx] += partial_sum; - }, - ); - - Ok(()) - } - - fn convert_to_state( - &self, - values: &[ArrayRef], - opt_filter: Option<&BooleanArray>, - ) -> Result> { - assert_eq!(values.len(), 1, "single argument to convert_to_state"); - let values = values[0].as_primitive::(); - let mut counts = Vec::with_capacity(values.len()); - let mut sums = Vec::with_capacity(values.len()); - let mut nulls = NullBufferBuilder::new(values.len()); - - for row in 0..values.len() { - if Self::filter_is_valid(opt_filter, row) && values.is_valid(row) { - counts.push(1); - sums.push(values.value(row)); - nulls.append_non_null(); - } else { - counts.push(0); - sums.push(0.0); - nulls.append_null(); - } - } - - let nulls = nulls.finish(); - Ok(vec![ - Arc::new(UInt64Array::new(counts.into(), nulls.clone())) as ArrayRef, - Arc::new(Float64Array::new(sums.into(), nulls)) as ArrayRef, - ]) - } - - fn supports_convert_to_state(&self) -> bool { - true - } - - fn size(&self) -> usize { - self.counts.len() * self.block_size * size_of::() - + self.sums.len() * self.block_size * size_of::() - + self.counts.allocated_size() - + self.sums.allocated_size() - + self.null_state.size() - } -} - -fn supports_blocked_avg_state(agg_expr: &Arc) -> Result { - if agg_expr.fun().name() != "avg" || agg_expr.is_distinct() { +pub(crate) fn can_use_blocked_hash_aggregate(agg: &AggregateExec) -> Result { + if !can_try_blocked_hash_aggregate(agg) { return Ok(false); } - let state_fields = agg_expr.state_fields()?; - Ok(matches!( - state_fields.as_slice(), - [count, sum] - if matches!(count.data_type(), DataType::UInt64) - && matches!(sum.data_type(), DataType::Float64) - && matches!(agg_expr.field().data_type(), DataType::Float64) - )) -} - -fn create_blocked_group_accumulator( - agg_expr: &Arc, - block_size: usize, -) -> Result>> { - if supports_blocked_avg_state(agg_expr)? { - Ok(Some(Box::new(BlockedAvgGroupsAccumulator::new(block_size)))) - } else { - Ok(None) - } -} - -fn create_blocked_group_accumulators( - aggr_expr: &[Arc], - block_size: usize, -) -> Result>>> { - let mut accumulators = Vec::with_capacity(aggr_expr.len()); - for agg_expr in aggr_expr { - let Some(accumulator) = create_blocked_group_accumulator(agg_expr, block_size)? - else { - return Ok(None); - }; - accumulators.push(accumulator); - } - Ok(Some(accumulators)) -} - -pub(crate) fn can_use_blocked_hash_aggregate(agg: &AggregateExec) -> Result { - if agg.group_by.has_grouping_set() || agg.group_by.groups().len() != 1 { + let input_schema = agg.input().schema(); + let group_schema = agg.group_by.group_schema(&input_schema)?; + let Some(group_values) = new_unordered_blocked_group_values(&group_schema, 1)? else { return Ok(false); - } + }; - let group_schema = agg.group_by.group_schema(&agg.input().schema())?; - if !supports_blocked_group_values(&group_schema) { + if !group_values.supports_blocked_emit() { return Ok(false); } for agg_expr in agg.aggr_expr.iter() { - if !supports_blocked_avg_state(agg_expr)? { + let Some(accumulator) = create_blocked_group_accumulator(agg_expr, 1)? else { + return Ok(false); + }; + if !accumulator.supports_blocked_emit() { return Ok(false); } } @@ -539,13 +196,24 @@ impl HashAggregateAccumulator { } fn evaluate_final(&mut self, emit_to: EmitTo) -> Result { + self.ensure_emit_supported(emit_to)?; self.accumulator.evaluate(emit_to) } fn state(&mut self, emit_to: EmitTo) -> Result> { + self.ensure_emit_supported(emit_to)?; self.accumulator.state(emit_to) } + fn ensure_emit_supported(&self, emit_to: EmitTo) -> Result<()> { + if matches!(emit_to, EmitTo::Block) && !self.accumulator.supports_blocked_emit() { + return internal_err!( + "EmitTo::Block is not supported by this GroupsAccumulator" + ); + } + Ok(()) + } + fn supports_convert_to_state(&self) -> bool { self.accumulator.supports_convert_to_state() } @@ -639,14 +307,14 @@ impl AggregateHashTable { )?; let group_schema = agg.group_by.group_schema(&input_schema)?; - let can_use_blocked_output = can_use_blocked_hash_aggregate(agg)?; + let can_try_blocked_output = can_try_blocked_hash_aggregate(agg); - let blocked_group_values = if can_use_blocked_output { + let blocked_group_values = if can_try_blocked_output { new_unordered_blocked_group_values(&group_schema, batch_size)? } else { None }; - let blocked_accumulators = if can_use_blocked_output { + let blocked_accumulators = if can_try_blocked_output { create_blocked_group_accumulators(&agg.aggr_expr, batch_size)? } else { None @@ -654,7 +322,10 @@ impl AggregateHashTable { let (group_values, accumulator_impls, blocked_output) = match (blocked_group_values, blocked_accumulators) { - (Some(group_values), Some(accumulators)) => { + (Some(group_values), Some(accumulators)) + if group_values.supports_blocked_emit() + && accumulators.iter().all(|acc| acc.supports_blocked_emit()) => + { (group_values, accumulators, true) } _ => { @@ -1355,7 +1026,7 @@ mod tests { use super::*; use arrow::array::Int64Array; use arrow::array::types::Int64Type; - use arrow::datatypes::{Field, Schema}; + use arrow::datatypes::{DataType, Field, Schema}; #[test] fn blocked_group_values_emit_blocks() -> Result<()> { @@ -1398,79 +1069,37 @@ mod tests { } #[test] - fn blocked_avg_state_emit_blocks() -> Result<()> { - let mut accumulator = BlockedAvgGroupsAccumulator::new(3); - let values: ArrayRef = Arc::new(Float64Array::from(vec![ - Some(1.0), - Some(2.0), - None, - Some(4.0), - Some(5.0), - Some(6.0), - Some(7.0), - ])); - let group_indices = vec![0, 1, 2, 3, 4, 5, 6]; - - accumulator.update_batch(&[values], &group_indices, None, 7)?; - - let state = accumulator.state(EmitTo::Block)?; - let counts = state[0].as_primitive::(); - let sums = state[1].as_primitive::(); - assert_eq!(counts.values().as_ref(), &[1, 1, 0]); - assert_eq!(sums.values().as_ref(), &[1.0, 2.0, 0.0]); - assert!(counts.is_null(2)); - assert!(sums.is_null(2)); - - let state = accumulator.state(EmitTo::Block)?; - assert_eq!( - state[0].as_primitive::().values().as_ref(), - &[1, 1, 1] - ); - assert_eq!( - state[1].as_primitive::().values().as_ref(), - &[4.0, 5.0, 6.0] - ); + fn blocked_group_values_support_primitive_keys() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "user_id", + DataType::UInt32, + false, + )])); + let group_values = + new_unordered_blocked_group_values(&schema, 3)?.expect("blocked UInt32"); - let state = accumulator.state(EmitTo::Block)?; - assert_eq!( - state[0].as_primitive::().values().as_ref(), - &[1] - ); - assert_eq!( - state[1].as_primitive::().values().as_ref(), - &[7.0] - ); + assert!(group_values.supports_blocked_emit()); Ok(()) } #[test] - fn blocked_avg_merge_evaluate_blocks() -> Result<()> { - let mut accumulator = BlockedAvgGroupsAccumulator::new(2); - let counts: ArrayRef = Arc::new(UInt64Array::from(vec![2, 1, 3, 4, 5])); - let sums: ArrayRef = - Arc::new(Float64Array::from(vec![10.0, 4.0, 9.0, 20.0, 15.0])); - let group_indices = vec![0, 1, 2, 3, 4]; - - accumulator.merge_batch(&[counts, sums], &group_indices, None, 5)?; - - let output = accumulator.evaluate(EmitTo::Block)?; - assert_eq!( - output.as_primitive::().values().as_ref(), - &[5.0, 4.0] - ); + fn non_blocked_group_values_reject_block_emit() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "user_id", + DataType::Int64, + false, + )])); + let mut group_values = new_unordered_group_values(schema)?; - let output = accumulator.evaluate(EmitTo::Block)?; - assert_eq!( - output.as_primitive::().values().as_ref(), - &[3.0, 5.0] - ); + let error = group_values + .emit(EmitTo::Block) + .expect_err("non-blocked group values should reject EmitTo::Block"); - let output = accumulator.evaluate(EmitTo::Block)?; - assert_eq!( - output.as_primitive::().values().as_ref(), - &[3.0] - ); + assert!(matches!( + error, + datafusion_common::DataFusionError::Internal(_) + )); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 1164fb37b384..a9d3ed6e55c3 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -713,6 +713,47 @@ pub(crate) fn create_group_accumulator( } } +/// Create a blocked accumulator for `agg_expr`, if the aggregate supports one. +pub(crate) fn create_blocked_group_accumulator( + agg_expr: &Arc, + block_size: usize, +) -> Result>> { + if !agg_expr.groups_accumulator_supported() { + return Ok(None); + } + + let accumulator = agg_expr.create_groups_accumulator()?; + let Some(blocked_accumulator) = accumulator.create_blocked_accumulator(block_size)? + else { + return Ok(None); + }; + + if !blocked_accumulator.supports_blocked_emit() { + return internal_err!( + "blocked accumulator for {} does not support blocked emit", + agg_expr.name() + ); + } + + Ok(Some(blocked_accumulator)) +} + +/// Create blocked accumulators for every aggregate expression, if all support it. +pub(crate) fn create_blocked_group_accumulators( + aggr_expr: &[Arc], + block_size: usize, +) -> Result>>> { + let mut accumulators = Vec::with_capacity(aggr_expr.len()); + for agg_expr in aggr_expr { + let Some(accumulator) = create_blocked_group_accumulator(agg_expr, block_size)? + else { + return Ok(None); + }; + accumulators.push(accumulator); + } + Ok(Some(accumulators)) +} + impl Stream for GroupedHashAggregateStream { type Item = Result; From 1b55110a7a1fdd58886cbbcd4b5e0d2b1c3143e5 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Tue, 2 Jun 2026 14:07:14 +0800 Subject: [PATCH 7/8] cleanup --- datafusion/core/tests/dataframe/mod.rs | 156 +++++++++--------- .../enforce_distribution.rs | 128 +++++++------- .../physical_optimizer/filter_pushdown.rs | 8 +- .../partition_statistics.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 8 +- .../src/aggregate/count_distinct/groups.rs | 22 ++- .../groups_accumulator/accumulate.rs | 3 +- .../aggregate/groups_accumulator/bool_op.rs | 11 +- .../aggregate/groups_accumulator/prim_op.rs | 8 +- .../functions-aggregate/src/array_agg.rs | 16 +- datafusion/functions-aggregate/src/average.rs | 30 +++- .../functions-aggregate/src/first_last.rs | 9 +- .../src/first_last/state.rs | 3 +- .../src/min_max/min_max_bytes.rs | 9 +- .../src/min_max/min_max_struct.rs | 9 +- .../physical-plan/src/aggregates/mod.rs | 3 +- .../physical-plan/src/aggregates/order/mod.rs | 3 +- .../sqllogictest/test_files/aggregate.slt | 41 ++--- .../test_files/aggregate_repartition.slt | 8 +- .../test_files/aggregates_simplify.slt | 8 +- .../test_files/aggregates_topk.slt | 16 +- .../sqllogictest/test_files/async_udf.slt | 4 +- .../sqllogictest/test_files/clickbench.slt | 144 ++++++++-------- .../test_files/count_star_rule.slt | 8 +- .../sqllogictest/test_files/distinct_on.slt | 4 +- .../sqllogictest/test_files/group_by.slt | 56 +++---- .../test_files/grouping_set_repartition.slt | 14 +- .../test_files/information_schema.slt | 2 +- .../sqllogictest/test_files/join.slt.part | 4 +- datafusion/sqllogictest/test_files/joins.slt | 12 +- .../sqllogictest/test_files/lateral_join.slt | 4 +- datafusion/sqllogictest/test_files/limit.slt | 12 +- datafusion/sqllogictest/test_files/order.slt | 20 +-- .../test_files/preserve_file_partitioning.slt | 16 +- .../test_files/projection_pushdown.slt | 4 +- .../test_files/push_down_filter_parquet.slt | 4 +- .../push_down_filter_regression.slt | 6 +- .../sqllogictest/test_files/qualify.slt | 8 +- .../sqllogictest/test_files/repartition.slt | 8 +- .../repartition_subset_satisfaction.slt | 8 +- datafusion/sqllogictest/test_files/select.slt | 4 +- .../sqllogictest/test_files/subquery.slt | 16 +- .../sqllogictest/test_files/subquery_sort.slt | 4 +- .../test_files/tpch/plans/q1.slt.part | 4 +- .../test_files/tpch/plans/q10.slt.part | 4 +- .../test_files/tpch/plans/q11.slt.part | 4 +- .../test_files/tpch/plans/q12.slt.part | 4 +- .../test_files/tpch/plans/q13.slt.part | 4 +- .../test_files/tpch/plans/q15.slt.part | 8 +- .../test_files/tpch/plans/q16.slt.part | 8 +- .../test_files/tpch/plans/q17.slt.part | 4 +- .../test_files/tpch/plans/q18.slt.part | 4 +- .../test_files/tpch/plans/q2.slt.part | 4 +- .../test_files/tpch/plans/q20.slt.part | 4 +- .../test_files/tpch/plans/q21.slt.part | 4 +- .../test_files/tpch/plans/q22.slt.part | 4 +- .../test_files/tpch/plans/q4.slt.part | 4 +- .../test_files/tpch/plans/q5.slt.part | 4 +- .../test_files/tpch/plans/q7.slt.part | 4 +- .../test_files/tpch/plans/q8.slt.part | 4 +- .../test_files/tpch/plans/q9.slt.part | 4 +- datafusion/sqllogictest/test_files/union.slt | 40 ++--- datafusion/sqllogictest/test_files/window.slt | 6 +- 63 files changed, 531 insertions(+), 460 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 61bc0e756039..0ced83f7b95f 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -3001,42 +3001,42 @@ async fn test_count_wildcard_on_sort() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+----------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+----------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) ASC NULLS LAST | - | | Projection: t1.b, count(Int64(1)) AS count(*) | - | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] | - | | TableScan: t1 projection=[b] | - | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | - | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | - | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*)] | - | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+----------------------------------------------------------------------------------------------------------------------------+ + +---------------+------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+------------------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) ASC NULLS LAST | + | | Projection: t1.b, count(Int64(1)) AS count(*) | + | | Aggregate: groupBy=[[t1.b]], aggr=[[count(Int64(1))]] | + | | TableScan: t1 projection=[b] | + | physical_plan | SortPreservingMergeExec: [count(*)@1 ASC NULLS LAST] | + | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | + | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*)] | + | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+------------------------------------------------------------------------------------+ " ); assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+-------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+-------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Sort: count(*) AS 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(1) as count(*)], stream=PartialFinalHashAggregateStream | - | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(1) as count(*)], stream=RawPartialHashAggregateStream | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-------------------------------------------------------------------------------------------------------------------------------+ + +---------------+---------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+---------------------------------------------------------------------------------------+ + | logical_plan | Sort: count(*) AS 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(1) as count(*)] | + | | RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[b@0 as b], aggr=[count(1) as count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+---------------------------------------------------------------------------------------+ " ); Ok(()) @@ -3433,30 +3433,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+--------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+--------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+--------------------------------------------------------------------------------------------------------------------------------+ + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ " ); @@ -3488,30 +3488,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+-------------------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+-------------------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(1) as count(*)], stream=PartialFinalHashAggregateStream | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(1) as count(*)], stream=RawPartialHashAggregateStream | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+-------------------------------------------------------------------------------------------------------------------------------------+ + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(1) as count(*)] | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(1) as count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 539a0cbbcb69..426e1fa745e5 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1047,14 +1047,14 @@ fn join_after_agg_alias() -> Result<()> { plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)] - AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream + 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " @@ -1105,14 +1105,14 @@ fn hash_join_key_ordering() -> Result<()> { @r" 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=RawPartialHashAggregateStream + 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " @@ -1829,16 +1829,16 @@ 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=RawPartialHashAggregateStream + 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -1850,16 +1850,16 @@ 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=RawPartialHashAggregateStream + 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -1937,17 +1937,17 @@ fn union_to_interleave() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[], stream=PartialFinalHashAggregateStream - AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream + 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream + 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 "); @@ -1984,18 +1984,18 @@ fn union_not_to_interleave() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20 - AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[] UnionExec - AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream + 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=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[], stream=RawPartialHashAggregateStream + 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 "); @@ -2014,9 +2014,9 @@ fn added_repartition_to_single_partition() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet "); @@ -2035,9 +2035,9 @@ fn repartition_deepest_node() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2139,9 +2139,9 @@ fn repartition_ignores_limit() -> Result<()> { let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -2589,9 +2589,9 @@ fn parallelization_single_partition() -> Result<()> { test_config.to_plan(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + 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 "); let plan_parquet_sort = test_config.to_plan(plan_parquet, &SORT_DISTRIB_DISTRIB); @@ -2601,9 +2601,9 @@ fn parallelization_single_partition() -> Result<()> { let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + 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 "); let plan_csv_sort = test_config.to_plan(plan_csv, &SORT_DISTRIB_DISTRIB); @@ -2701,9 +2701,9 @@ fn parallelization_compressed_csv() -> Result<()> { // Compressed files cannot be partitioned assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); @@ -2711,9 +2711,9 @@ fn parallelization_compressed_csv() -> Result<()> { // Uncompressed files can be partitioned assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + 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 "); } @@ -2740,9 +2740,9 @@ fn parallelization_two_partitions() -> Result<()> { test_config.to_plan(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet "); // Plan already has two partitions @@ -2752,9 +2752,9 @@ fn parallelization_two_partitions() -> Result<()> { // Test: with csv let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); // Plan already has two partitions @@ -2780,9 +2780,9 @@ fn parallelization_two_partitions_into_four() -> Result<()> { // Multiple source files split across partitions assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet "); // Multiple source files split across partitions @@ -2793,9 +2793,9 @@ fn parallelization_two_partitions_into_four() -> Result<()> { let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false "); // Multiple source files split across partitions @@ -2920,9 +2920,9 @@ fn parallelization_ignores_limit() -> Result<()> { test_config.to_plan(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_parquet_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -2942,9 +2942,9 @@ fn parallelization_ignores_limit() -> Result<()> { let plan_csv_distrib = test_config.to_plan(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_csv_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -3363,9 +3363,9 @@ fn preserve_ordering_for_streaming_sorted_aggregate() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted, stream=GroupedHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted, stream=GroupedHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[COUNT(b)], ordering_mode=Sorted DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet "); @@ -3397,9 +3397,9 @@ fn preserve_ordering_for_streaming_partially_sorted_aggregate() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]), stream=GroupedHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]) RepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC - AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]), stream=GroupedHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[COUNT(c)], ordering_mode=PartiallySorted([0]) DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet "); @@ -3633,8 +3633,8 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=FinalPartitioned, 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 "); let plan_sort = test_config.to_plan(physical_plan, &SORT_DISTRIB_DISTRIB); @@ -3663,11 +3663,11 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let plan_distrib = test_config.to_plan(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!(plan_distrib, @r" - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream - AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 - AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet "); diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 92c93547c8d1..b420326596d0 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -1848,7 +1848,7 @@ fn test_pushdown_filter_on_non_first_grouping_column() { OptimizationTest: input: - FilterExec: b@1 = bar - - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], stream=PartialFinalHashAggregateStream + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: @@ -2114,7 +2114,7 @@ fn test_pushdown_through_aggregate_with_reordered_input_columns() { OptimizationTest: input: - FilterExec: b@1 = bar - - AggregateExec: mode=Final, gby=[a@1 as a, b@2 as b], aggr=[cnt], stream=PartialFinalHashAggregateStream + - AggregateExec: mode=Final, gby=[a@1 as a, b@2 as b], aggr=[cnt] - ProjectionExec: expr=[c@2 as c, a@0 as a, b@1 as b] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: @@ -2291,12 +2291,12 @@ fn test_pushdown_with_computed_grouping_key() { @r" OptimizationTest: input: - - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt], stream=PartialFinalHashAggregateStream + - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt] - FilterExec: c@2 > 5 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true output: Ok: - - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt], stream=PartialFinalHashAggregateStream + - AggregateExec: mode=Final, gby=[c@2 + 1 as c_plus_1], aggr=[cnt] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=c@2 > 5 " ); diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 0ee2ed8ba215..d06e506abfeb 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -826,7 +826,7 @@ mod test { let plan_string = get_plan_string(&aggregate_exec_partial).swap_remove(0); assert_snapshot!( plan_string, - @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)], stream=RawPartialHashAggregateStream" + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" ); let p0_statistics = aggregate_exec_partial.partition_statistics(Some(0))?; @@ -922,7 +922,7 @@ mod test { let agg_plan = get_plan_string(&agg_partial).remove(0); assert_snapshot!( agg_plan, - @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)], stream=RawPartialHashAggregateStream" + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" ); let empty_stat = Statistics { diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index f13a1a7c0de8..17e3dba14b90 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -69,8 +69,7 @@ async fn explain_analyze_baseline_metrics() { assert_metrics!( &formatted, "AggregateExec: mode=Partial, gby=[c1@0 as c1]", - "stream=RawPartialHashAggregateStream", - "skipped_aggregation_rows=0" + "reduction_factor=5.05% (5/99)" ); { @@ -84,7 +83,6 @@ async fn explain_analyze_baseline_metrics() { assert_metrics!( &formatted, "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1]", - "stream=PartialFinalHashAggregateStream", "metrics=[output_rows=5, elapsed_compute=", "output_bytes=", expected_batch_count_after_repartition @@ -778,9 +776,9 @@ 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)], stream=PartialFinalHashAggregateStream + AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)] 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)], stream=RawPartialHashAggregateStream + AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)] FilterExec: c12@1 < 10 RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true diff --git a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs index a75363e59217..75c9cfc0f9fa 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs @@ -20,8 +20,8 @@ use arrow::array::{ }; use arrow::buffer::{OffsetBuffer, ScalarBuffer}; use arrow::datatypes::{ArrowPrimitiveType, Field}; -use datafusion_common::HashSet; use datafusion_common::hash_utils::RandomState; +use datafusion_common::{HashSet, internal_err}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use std::hash::Hash; use std::mem::size_of; @@ -82,12 +82,19 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> datafusion_common::Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by PrimitiveDistinctCountGroupsAccumulator" + ); + } + let counts = emit_to.take_needed(&mut self.counts); match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { self.seen.clear(); } + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) => { let mut remaining = HashSet::default(); for (group_idx, value) in self.seen.drain() { @@ -103,8 +110,15 @@ where } fn state(&mut self, emit_to: EmitTo) -> datafusion_common::Result> { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by PrimitiveDistinctCountGroupsAccumulator" + ); + } + let num_emitted = match emit_to { - EmitTo::All | EmitTo::Block => self.counts.len(), + EmitTo::All => self.counts.len(), + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) => n, }; @@ -120,7 +134,7 @@ where let mut all_values = vec![T::Native::default(); total as usize]; let mut cursors: Vec = offsets[..num_emitted].to_vec(); - if matches!(emit_to, EmitTo::All | EmitTo::Block) { + if matches!(emit_to, EmitTo::All) { for (group_idx, value) in self.seen.drain() { let pos = cursors[group_idx] as usize; all_values[pos] = value; diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 235ad10a8a4c..b650cc5c3702 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -296,7 +296,7 @@ impl NullState { /// resets the internal state appropriately pub fn build(&mut self, emit_to: EmitTo) -> Option { match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { let old_seen = std::mem::take(&mut self.seen_values); match old_seen { SeenValues::All { .. } => None, @@ -305,6 +305,7 @@ impl NullState { } } } + EmitTo::Block => unreachable!("handled by caller"), EmitTo::First(n) => match &mut self.seen_values { SeenValues::All { num_values } => { *num_values = num_values.saturating_sub(n); diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs index 3929d8edac59..651a00f4bf46 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use crate::aggregate::groups_accumulator::nulls::filtered_null_mask; use arrow::array::{ArrayRef, AsArray, BooleanArray, BooleanBufferBuilder}; use arrow::buffer::BooleanBuffer; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use super::accumulate::NullState; @@ -105,10 +105,17 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by BooleanGroupsAccumulator" + ); + } + let values = self.values.finish(); let values = match emit_to { - EmitTo::All | EmitTo::Block => values, + EmitTo::All => values, + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) => { let first_n: BooleanBuffer = values.iter().take(n).collect(); // put n+1 back into self.values diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs index a81b89e1e46f..2e1e0cdf14fd 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs @@ -23,7 +23,7 @@ use arrow::buffer::NullBuffer; use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; -use datafusion_common::{DataFusionError, Result, internal_datafusion_err}; +use datafusion_common::{DataFusionError, Result, internal_datafusion_err, internal_err}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use super::accumulate::NullState; @@ -116,6 +116,12 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by PrimitiveGroupsAccumulator" + ); + } + let values = emit_to.take_needed(&mut self.values); let nulls = self.null_state.build(emit_to); let values = PrimitiveArray::::new(values.into(), nulls) // no copy diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index bf7e18722b54..bc7206420f30 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -34,7 +34,9 @@ use datafusion_common::cast::as_list_array; use datafusion_common::utils::{ SingleRowListArrayBuilder, compare_rows, get_row_at_idx, take_function_args, }; -use datafusion_common::{Result, ScalarValue, assert_eq_or_internal_err, exec_err}; +use datafusion_common::{ + Result, ScalarValue, assert_eq_or_internal_err, exec_err, internal_err, +}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ @@ -651,8 +653,15 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { /// entries into group order, then calls `interleave` to gather /// the values into a flat array that backs the output `ListArray`. fn evaluate(&mut self, emit_to: EmitTo) -> Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by ArrayAggGroupsAccumulator" + ); + } + let emit_groups = match emit_to { - EmitTo::All | EmitTo::Block => self.num_groups, + EmitTo::All => self.num_groups, + EmitTo::Block => unreachable!("handled above"), EmitTo::First(n) => n, }; @@ -711,7 +720,8 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { // Step 4: Release state for emitted groups. match emit_to { - EmitTo::All | EmitTo::Block => self.clear_state(), + EmitTo::All => self.clear_state(), + EmitTo::Block => unreachable!("handled above"), EmitTo::First(_) => self.compact_retained_state(emit_groups)?, } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index cffa0f528ca9..8a0d75a6397a 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -33,7 +33,7 @@ use arrow::datatypes::{ DurationSecondType, Field, FieldRef, Float64Type, TimeUnit, UInt64Type, i256, }; use datafusion_common::types::{NativeType, logical_float64}; -use datafusion_common::{Result, ScalarValue, exec_err, not_impl_err}; +use datafusion_common::{Result, ScalarValue, exec_err, internal_err, not_impl_err}; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; @@ -792,7 +792,21 @@ where avg_fn: F, } -/// Avg accumulator that stores its per-group state in fixed-size blocks. +/// Avg accumulator for primitive types. +/// +/// It is mostly equivalent to `AvgGroupsAccumulator`, but uses a block-based +/// allocation policy for internal buffers: +/// +/// block_size: 100 +/// counts: [100 states], [100 states], [100 states], ... +/// sums : [100 states], [100 states], [100 states], ... +/// +/// The `group` index passed to `GroupsAccumulator` APIs is interpreted relative +/// to the accumulator's start. The block index and index within that block are +/// handled internally. +/// +/// For example, with `block_size` 100, group index 101 maps to block index +/// `101 / 100 == 1` and index within the block `101 % 100 == 1`. #[derive(Debug)] struct BlockedAvgGroupsAccumulator { block_size: usize, @@ -982,6 +996,12 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by AvgGroupsAccumulator" + ); + } + let counts = emit_to.take_needed(&mut self.counts); let sums = emit_to.take_needed(&mut self.sums); let nulls = self.null_state.build(emit_to); @@ -1023,6 +1043,12 @@ where // return arrays for sums and counts fn state(&mut self, emit_to: EmitTo) -> Result> { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by AvgGroupsAccumulator" + ); + } + let nulls = self.null_state.build(emit_to); let counts = emit_to.take_needed(&mut self.counts); diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index cc60c2e82b27..336b005d810c 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -441,7 +441,8 @@ impl FirstLastGroupsAccumulator { let result = emit_to.take_needed(&mut self.orderings); match emit_to { - EmitTo::All | EmitTo::Block => self.size_of_orderings = 0, + EmitTo::All => self.size_of_orderings = 0, + EmitTo::Block => unreachable!("handled by caller"), EmitTo::First(_) => { self.size_of_orderings -= result.iter().map(ScalarValue::size_of_vec).sum::() @@ -498,6 +499,12 @@ impl FirstLastGroupsAccumulator { &mut self, emit_to: EmitTo, ) -> Result<(ArrayRef, Vec>, BooleanBuffer)> { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by FirstLastGroupsAccumulator" + ); + } + emit_to.take_needed(&mut self.extreme_of_each_group_buf.0); self.extreme_of_each_group_buf .1 diff --git a/datafusion/functions-aggregate/src/first_last/state.rs b/datafusion/functions-aggregate/src/first_last/state.rs index e83b190468b7..1636bd460392 100644 --- a/datafusion/functions-aggregate/src/first_last/state.rs +++ b/datafusion/functions-aggregate/src/first_last/state.rs @@ -296,7 +296,8 @@ pub(crate) fn take_need( ) -> BooleanBuffer { let bool_buf = bool_buf_builder.finish(); match emit_to { - EmitTo::All | EmitTo::Block => bool_buf, + EmitTo::All => bool_buf, + EmitTo::Block => unreachable!("handled by caller"), EmitTo::First(n) => { // split off the first N values in seen_values // diff --git a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs index adcec89537ef..d10dffafc197 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -202,6 +202,12 @@ impl GroupsAccumulator for MinMaxBytesAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by MinMaxBytesAccumulator" + ); + } + let (data_capacity, min_maxes) = self.inner.emit_to(emit_to); // Convert the Vec of bytes to a vec of Strings (at no cost) @@ -488,12 +494,13 @@ impl MinMaxBytesState { /// - `min_maxes`: the actual min/max values for each group fn emit_to(&mut self, emit_to: EmitTo) -> (usize, Vec>>) { match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { ( std::mem::take(&mut self.total_data_bytes), // reset total bytes and min_max std::mem::take(&mut self.min_max), ) } + EmitTo::Block => unreachable!("handled by caller"), EmitTo::First(n) => { let first_min_maxes = split_vec_min_alloc(&mut self.min_max, n); let first_data_capacity: usize = first_min_maxes diff --git a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs index ec9d9e62b4b1..9229c7e25b8c 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs @@ -101,6 +101,12 @@ impl GroupsAccumulator for MinMaxStructAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { + if matches!(emit_to, EmitTo::Block) { + return internal_err!( + "EmitTo::Block is not supported by MinMaxStructAccumulator" + ); + } + let (_, min_maxes) = self.inner.emit_to(emit_to); let fields = match &self.inner.data_type { DataType::Struct(fields) => fields, @@ -277,12 +283,13 @@ impl MinMaxStructState { /// - `min_maxes`: the actual min/max values for each group fn emit_to(&mut self, emit_to: EmitTo) -> (usize, Vec>) { match emit_to { - EmitTo::All | EmitTo::Block => { + EmitTo::All => { ( std::mem::take(&mut self.total_data_bytes), // reset total bytes and min_max std::mem::take(&mut self.min_max), ) } + EmitTo::Block => unreachable!("handled by caller"), EmitTo::First(n) => { let first_min_maxes = split_vec_min_alloc(&mut self.min_max, n); let first_data_capacity: usize = first_min_maxes diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4553df59dc73..285440d7294b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1065,8 +1065,7 @@ impl AggregateExec { fn should_display_stream_name(&self, t: DisplayFormatType) -> bool { matches!(t, DisplayFormatType::Verbose) - || self.can_use_raw_partial_hash_stream() - || self.can_use_partial_final_hash_stream() + || self.uses_blocked_hash_stream_for_display() } /// Finds the DataType and SortDirection for this Aggregate, if there is one diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 9a23ec12ddb3..fe3ce714aa42 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -78,7 +78,8 @@ impl GroupOrdering { GroupOrdering::Partial(_) | GroupOrdering::Full(_) => { self.emit_to().map(|emit_to| match emit_to { EmitTo::First(max) => EmitTo::First(n.min(max)), - EmitTo::All | EmitTo::Block => EmitTo::First(n), + EmitTo::All => EmitTo::First(n), + EmitTo::Block => unreachable!("GroupOrdering does not emit blocks"), }) } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index c9df9c64a459..25b69d16dd03 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -69,19 +69,6 @@ statement ok CREATE TABLE test (c1 BIGINT,c2 BIGINT) as values (0,null), (1,1), (null,1), (3,2), (3,2) -# Regular indent EXPLAIN shows the concrete aggregate stream implementation. -statement ok -set datafusion.explain.format = 'indent'; - -query TT -EXPLAIN -SELECT value % 2 AS g, COUNT(*) -FROM generate_series(1, 3) -GROUP BY g; ----- -AggregateExec: mode=FinalPartitionedstream=PartialFinalHashAggregateStream -AggregateExec: mode=Partialstream=RawPartialHashAggregateStream - statement ok CREATE TABLE group_median_table_non_nullable ( col_group STRING NOT NULL, @@ -1685,9 +1672,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, gby=[alias1@0 as alias1], aggr=[], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[CAST(c@0 AS Float64) as alias1], aggr=[], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[CAST(c@0 AS Float64) as alias1], aggr=[] 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok @@ -6543,9 +6530,9 @@ logical_plan physical_plan 01)CoalescePartitionsExec: fetch=5 02)--AggregateExec: mode=SinglePartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] 04)------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 -05)--------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c1, c3], file_type=csv, has_header=true @@ -6625,9 +6612,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[4] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[c3@1 as c3] -07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=PartialFinalHashAggregateStream +07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 10)------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -6652,9 +6639,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(aggregate_test_100.c1)@2 as max(aggregate_test_100.c1), c2@0 as c2, c3@1 as c3] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[max(aggregate_test_100.c1)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[max(aggregate_test_100.c1)] 04)------CoalescePartitionsExec -05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true @@ -6676,9 +6663,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] -07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=PartialFinalHashAggregateStream +07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -6727,9 +6714,9 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c3] physical_plan 01)GlobalLimitExec: skip=0, fetch=5 -02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100_with_dates.csv]]}, projection=[c3], file_type=csv, has_header=true @@ -7781,9 +7768,9 @@ logical_plan 04)------TableScan: having_test projection=[v1, v2] physical_plan 01)FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] -02)--AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] 03)----RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/aggregate_repartition.slt b/datafusion/sqllogictest/test_files/aggregate_repartition.slt index b5ddbf02feaf..1f1e72681167 100644 --- a/datafusion/sqllogictest/test_files/aggregate_repartition.slt +++ b/datafusion/sqllogictest/test_files/aggregate_repartition.slt @@ -76,9 +76,9 @@ logical_plan 03)----TableScan: dim_csv projection=[env] physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] -02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=4 -04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.csv]]}, projection=[env], file_type=csv, has_header=true @@ -93,9 +93,9 @@ logical_plan 03)----TableScan: dim_parquet projection=[env] physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] -02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet # Verify the queries actually work and return the same results diff --git a/datafusion/sqllogictest/test_files/aggregates_simplify.slt b/datafusion/sqllogictest/test_files/aggregates_simplify.slt index 7e07cd24f4f9..c4055d17396c 100644 --- a/datafusion/sqllogictest/test_files/aggregates_simplify.slt +++ b/datafusion/sqllogictest/test_files/aggregates_simplify.slt @@ -170,9 +170,9 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(alias1), sum(alias2)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(alias1), sum(alias2)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[sum(__common_expr_1) as alias2], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[sum(__common_expr_1) as alias2] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as alias1], aggr=[sum(__common_expr_1) as alias2], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as alias1], aggr=[sum(__common_expr_1) as alias2] 08)--------------ProjectionExec: expr=[column1@0 + 1 as __common_expr_1] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -273,9 +273,9 @@ physical_plan 01)SortPreservingMergeExec: [column2@0 DESC NULLS LAST] 02)--SortExec: expr=[column2@0 DESC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[column2@0 as column2, sum(sum_simplify_t.column1)@1 + count(sum_simplify_t.column1)@2 as sum(sum_simplify_t.column1 + Int64(1)), sum(sum_simplify_t.column1)@1 + 2 * count(sum_simplify_t.column1)@2 as sum(sum_simplify_t.column1 + Int64(2))] -04)------AggregateExec: mode=FinalPartitioned, gby=[column2@0 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[column2@0 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)] 05)--------RepartitionExec: partitioning=Hash([column2@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[column2@1 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[column2@1 as column2], aggr=[sum(sum_simplify_t.column1), count(sum_simplify_t.column1)] 07)------------DataSourceExec: partitions=1, partition_sizes=[1] # Checks commutative forms of equivalent aggregate arguments are simplified consistently. diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 947a02ed690c..19ead8965ed0 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -45,9 +45,9 @@ 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)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TI @@ -122,9 +122,9 @@ 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)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -137,9 +137,9 @@ 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)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -152,9 +152,9 @@ 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)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TI diff --git a/datafusion/sqllogictest/test_files/async_udf.slt b/datafusion/sqllogictest/test_files/async_udf.slt index 07f985a99ce4..678b2f7d8b8d 100644 --- a/datafusion/sqllogictest/test_files/async_udf.slt +++ b/datafusion/sqllogictest/test_files/async_udf.slt @@ -56,9 +56,9 @@ logical_plan 04)------TableScan: data projection=[x] physical_plan 01)ProjectionExec: expr=[min(async_abs(data.x))@1 as min(async_abs(data.x))] -02)--AggregateExec: mode=FinalPartitioned, gby=[async_abs(data.x)@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[async_abs(data.x)@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))] 03)----RepartitionExec: partitioning=Hash([async_abs(data.x)@0], 4), input_partitions=4 -04)------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[__common_expr_1@0 as async_abs(data.x)], aggr=[min(async_abs(data.x))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------ProjectionExec: expr=[__async_fn_0@1 as __common_expr_1] 07)------------AsyncFuncExec: async_expr=[async_expr(name=__async_fn_0, expr=async_abs(x@0))] diff --git a/datafusion/sqllogictest/test_files/clickbench.slt b/datafusion/sqllogictest/test_files/clickbench.slt index 5155b5205f41..91463c9c2bff 100644 --- a/datafusion/sqllogictest/test_files/clickbench.slt +++ b/datafusion/sqllogictest/test_files/clickbench.slt @@ -147,9 +147,9 @@ 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=[], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[UserID@0 as alias1], aggr=[] 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet query I @@ -172,9 +172,9 @@ 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=[], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as alias1], aggr=[] 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet query I @@ -215,9 +215,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@1 DESC] 02)--SortExec: expr=[count(*)@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[AdvEngineID@0 as AdvEngineID, count(Int64(1))@1 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([AdvEngineID@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[AdvEngineID@0 as AdvEngineID], aggr=[count(Int64(1))] 07)------------FilterExec: AdvEngineID@0 != 0 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[AdvEngineID], file_type=parquet, predicate=AdvEngineID@40 != 0, pruning_predicate=AdvEngineID_null_count@2 != row_count@3 AND (AdvEngineID_min@0 != 0 OR 0 != AdvEngineID_max@1), required_guarantees=[AdvEngineID not in (0)] @@ -241,12 +241,12 @@ physical_plan 01)SortPreservingMergeExec: [u@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[RegionID@0 as RegionID, count(alias1)@1 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] 05)--------RepartitionExec: partitioning=Hash([RegionID@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)], stream=RawPartialHashAggregateStream -07)------------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[count(alias1)] +07)------------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID, alias1@1 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=Hash([RegionID@0, alias1@1], 4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID, UserID@1 as alias1], aggr=[] 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[RegionID, UserID], file_type=parquet query II rowsort @@ -271,9 +271,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[RegionID@0 as RegionID, sum(hits.AdvEngineID)@1 as sum(hits.AdvEngineID), count(Int64(1))@2 as c, avg(hits.ResolutionWidth)@3 as avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] -04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] 05)--------RepartitionExec: partitioning=Hash([RegionID@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[RegionID@0 as RegionID], aggr=[sum(hits.AdvEngineID), count(Int64(1)), avg(hits.ResolutionWidth), count(DISTINCT hits.UserID)] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[RegionID, UserID, ResolutionWidth, AdvEngineID], file_type=parquet query IIIRI rowsort @@ -300,12 +300,12 @@ physical_plan 01)SortPreservingMergeExec: [u@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[MobilePhoneModel@0 as MobilePhoneModel, count(alias1)@1 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] 05)--------RepartitionExec: partitioning=Hash([MobilePhoneModel@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)], stream=RawPartialHashAggregateStream -07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[MobilePhoneModel@0 as MobilePhoneModel], aggr=[count(alias1)] +07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhoneModel@0 as MobilePhoneModel, alias1@1 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=Hash([MobilePhoneModel@0, alias1@1], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[MobilePhoneModel@1 as MobilePhoneModel, UserID@0 as alias1], aggr=[] 10)------------------FilterExec: MobilePhoneModel@1 != 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] @@ -330,12 +330,12 @@ physical_plan 01)SortPreservingMergeExec: [u@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, count(alias1)@2 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] 05)--------RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)], stream=RawPartialHashAggregateStream -07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel], aggr=[count(alias1)] +07)------------AggregateExec: mode=FinalPartitioned, gby=[MobilePhone@0 as MobilePhone, MobilePhoneModel@1 as MobilePhoneModel, alias1@2 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=Hash([MobilePhone@0, MobilePhoneModel@1, alias1@2], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[MobilePhone@1 as MobilePhone, MobilePhoneModel@2 as MobilePhoneModel, UserID@0 as alias1], aggr=[] 10)------------------FilterExec: MobilePhoneModel@2 != 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, MobilePhone, MobilePhoneModel], file_type=parquet, predicate=MobilePhoneModel@34 != , pruning_predicate=MobilePhoneModel_null_count@2 != row_count@3 AND (MobilePhoneModel_min@0 != OR != MobilePhoneModel_max@1), required_guarantees=[MobilePhoneModel not in ()] @@ -359,9 +359,9 @@ physical_plan 01)SortPreservingMergeExec: [c@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(Int64(1))@1 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(Int64(1))] 07)------------FilterExec: SearchPhrase@0 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -386,12 +386,12 @@ physical_plan 01)SortPreservingMergeExec: [u@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[u@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, count(alias1)@1 as u] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)], stream=RawPartialHashAggregateStream -07)------------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@0 as SearchPhrase], aggr=[count(alias1)] +07)------------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase, alias1@1 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=Hash([SearchPhrase@0, alias1@1], 4), input_partitions=4 -09)----------------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase, UserID@0 as alias1], aggr=[] 10)------------------FilterExec: SearchPhrase@1 != 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -415,9 +415,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([SearchEngineID@0, SearchPhrase@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@0 as SearchEngineID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 07)------------FilterExec: SearchPhrase@1 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -440,9 +440,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(*)@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[UserID@0 as UserID, count(Int64(1))@1 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([UserID@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(Int64(1))] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID], file_type=parquet query II rowsort @@ -468,9 +468,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(*)@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet query ITI rowsort @@ -495,9 +495,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase, count(Int64(1))@2 as count(*)] 02)--CoalescePartitionsExec: fetch=10 -03)----AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 04)------RepartitionExec: partitioning=Hash([UserID@0, SearchPhrase@1], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[UserID@0 as UserID, SearchPhrase@1 as SearchPhrase], aggr=[count(Int64(1))] 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[UserID, SearchPhrase], file_type=parquet query ITI rowsort @@ -523,9 +523,9 @@ physical_plan 01)SortPreservingMergeExec: [count(*)@3 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(*)@3 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as m, SearchPhrase@2 as SearchPhrase, count(Int64(1))@3 as count(*)] -04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1 as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([UserID@0, date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime))@1, SearchPhrase@2], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[UserID@1 as UserID, date_part(MINUTE, to_timestamp_seconds(EventTime@0)) as date_part(Utf8("MINUTE"),to_timestamp_seconds(hits.EventTime)), SearchPhrase@2 as SearchPhrase], aggr=[count(Int64(1))] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, UserID, SearchPhrase], file_type=parquet query IITI rowsort @@ -599,9 +599,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, min(hits.URL)@1 as min(hits.URL), count(Int64(1))@2 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@1 as SearchPhrase], aggr=[min(hits.URL), count(Int64(1))] 07)------------FilterExec: URL@0 LIKE %google% AND SearchPhrase@1 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL, SearchPhrase], file_type=parquet, predicate=URL@13 LIKE %google% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@4 != row_count@5 AND (SearchPhrase_min@2 != OR != SearchPhrase_max@3), required_guarantees=[SearchPhrase not in ()] @@ -625,9 +625,9 @@ physical_plan 01)SortPreservingMergeExec: [c@3 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@3 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase, min(hits.URL)@1 as min(hits.URL), min(hits.Title)@2 as min(hits.Title), count(Int64(1))@3 as c, count(DISTINCT hits.UserID)@4 as count(DISTINCT hits.UserID)] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchPhrase@0 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] 05)--------RepartitionExec: partitioning=Hash([SearchPhrase@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@3 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[SearchPhrase@3 as SearchPhrase], aggr=[min(hits.URL), min(hits.Title), count(Int64(1)), count(DISTINCT hits.UserID)] 07)------------FilterExec: Title@0 LIKE %Google% AND URL@2 NOT LIKE %.google.% AND SearchPhrase@3 != 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Title, UserID, URL, SearchPhrase], file_type=parquet, predicate=Title@2 LIKE %Google% AND URL@13 NOT LIKE %.google.% AND SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@6 != row_count@7 AND (SearchPhrase_min@4 != OR != SearchPhrase_max@5), required_guarantees=[SearchPhrase not in ()] @@ -741,9 +741,9 @@ physical_plan 02)--SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CounterID@0 as CounterID, avg(length(hits.URL))@1 as l, count(Int64(1))@2 as c] 04)------FilterExec: count(Int64(1))@2 > 100000 -05)--------AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([CounterID@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[CounterID@0 as CounterID], aggr=[avg(length(hits.URL)), count(Int64(1))] 08)--------------FilterExec: URL@1 != 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[CounterID, URL], file_type=parquet, predicate=URL@13 != , pruning_predicate=URL_null_count@2 != row_count@3 AND (URL_min@0 != OR != URL_max@1), required_guarantees=[URL not in ()] @@ -769,9 +769,9 @@ physical_plan 02)--SortExec: TopK(fetch=25), expr=[l@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as k, avg(length(hits.Referer))@1 as l, count(Int64(1))@2 as c, min(hits.Referer)@3 as min(hits.Referer)] 04)------FilterExec: count(Int64(1))@2 > 100000 -05)--------AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0 as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] 06)----------RepartitionExec: partitioning=Hash([regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[regexp_replace(Referer@0, ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[regexp_replace(Referer@0, ^https?://(?:www\.)?([^/]+)/.*$, \1) as regexp_replace(hits.Referer,Utf8("^https?://(?:www\.)?([^/]+)/.*$"),Utf8("\1"))], aggr=[avg(length(hits.Referer)), count(Int64(1)), min(hits.Referer)] 08)--------------FilterExec: Referer@0 != 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Referer], file_type=parquet, predicate=Referer@14 != , pruning_predicate=Referer_null_count@2 != row_count@3 AND (Referer_min@0 != OR != Referer_max@1), required_guarantees=[Referer not in ()] @@ -817,9 +817,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] -04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[SearchEngineID@0 as SearchEngineID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 05)--------RepartitionExec: partitioning=Hash([SearchEngineID@0, ClientIP@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[SearchEngineID@3 as SearchEngineID, ClientIP@0 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 07)------------FilterExec: SearchPhrase@4 != , projection=[ClientIP@0, IsRefresh@1, ResolutionWidth@2, SearchEngineID@3] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[ClientIP, IsRefresh, ResolutionWidth, SearchEngineID, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -844,9 +844,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[WatchID@0 as WatchID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] -04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 05)--------RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 07)------------FilterExec: SearchPhrase@4 != , projection=[WatchID@0, ClientIP@1, IsRefresh@2, ResolutionWidth@3] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != , pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] @@ -869,9 +869,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[WatchID@0 as WatchID, ClientIP@1 as ClientIP, count(Int64(1))@2 as c, sum(hits.IsRefresh)@3 as sum(hits.IsRefresh), avg(hits.ResolutionWidth)@4 as avg(hits.ResolutionWidth)] -04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 05)--------RepartitionExec: partitioning=Hash([WatchID@0, ClientIP@1], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[WatchID@0 as WatchID, ClientIP@1 as ClientIP], aggr=[count(Int64(1)), sum(hits.IsRefresh), avg(hits.ResolutionWidth)] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, ClientIP, IsRefresh, ResolutionWidth], file_type=parquet query IIIIR rowsort @@ -902,9 +902,9 @@ physical_plan 01)SortPreservingMergeExec: [c@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet query TI rowsort @@ -931,9 +931,9 @@ physical_plan 01)SortPreservingMergeExec: [c@2 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[c@2 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[1 as Int64(1), URL@0 as URL, count(Int64(1))@1 as c] -04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] 07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[URL], file_type=parquet query ITI rowsort @@ -962,9 +962,9 @@ physical_plan 02)--SortExec: TopK(fetch=10), expr=[c@4 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[ClientIP@1 as ClientIP, __common_expr_1@0 - 1 as hits.ClientIP - Int64(1), __common_expr_1@0 - 2 as hits.ClientIP - Int64(2), __common_expr_1@0 - 3 as hits.ClientIP - Int64(3), count(Int64(1))@2 as c] 04)------ProjectionExec: expr=[CAST(ClientIP@0 AS Int64) as __common_expr_1, ClientIP@0 as ClientIP, count(Int64(1))@1 as count(Int64(1))] -05)--------AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([ClientIP@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[ClientIP@0 as ClientIP], aggr=[count(Int64(1))] 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[ClientIP], file_type=parquet query IIIII rowsort @@ -991,9 +991,9 @@ physical_plan 01)SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] -04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] 07)------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND URL@2 != , projection=[URL@2] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND URL@13 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND URL_null_count@15 != row_count@3 AND (URL_min@13 != OR != URL_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URL not in ()] @@ -1018,9 +1018,9 @@ physical_plan 01)SortPreservingMergeExec: [pageviews@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[pageviews@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[Title@0 as Title, count(Int64(1))@1 as pageviews] -04)------AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[Title@0 as Title], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([Title@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[Title@0 as Title], aggr=[count(Int64(1))] 07)------------FilterExec: CounterID@2 = 62 AND EventDate@1 >= 15887 AND EventDate@1 <= 15917 AND DontCountHits@4 = 0 AND IsRefresh@3 = 0 AND Title@0 != , projection=[Title@0] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[Title, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND DontCountHits@61 = 0 AND IsRefresh@15 = 0 AND Title@2 != , pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND DontCountHits_null_count@9 != row_count@3 AND DontCountHits_min@7 <= 0 AND 0 <= DontCountHits_max@8 AND IsRefresh_null_count@12 != row_count@3 AND IsRefresh_min@10 <= 0 AND 0 <= IsRefresh_max@11 AND Title_null_count@15 != row_count@3 AND (Title_min@13 != OR != Title_max@14), required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), Title not in ()] @@ -1047,9 +1047,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@1 DESC], fetch=1010 03)----SortExec: TopK(fetch=1010), expr=[pageviews@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[URL@0 as URL, count(Int64(1))@1 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[URL@0 as URL], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([URL@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[URL@0 as URL], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@3 = 0 AND IsLink@4 != 0 AND IsDownload@5 = 0, projection=[URL@2] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, IsRefresh, IsLink, IsDownload], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND IsRefresh@15 = 0 AND IsLink@52 != 0 AND IsDownload@53 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND IsLink_null_count@12 != row_count@3 AND (IsLink_min@10 != 0 OR 0 != IsLink_max@11) AND IsDownload_null_count@15 != row_count@3 AND IsDownload_min@13 <= 0 AND 0 <= IsDownload_max@14, required_guarantees=[CounterID in (62), IsDownload in (0), IsLink not in (0), IsRefresh in (0)] @@ -1076,9 +1076,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@5 DESC], fetch=1010 03)----SortExec: TopK(fetch=1010), expr=[pageviews@5 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as src, URL@4 as dst, count(Int64(1))@5 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[TraficSourceID@0 as TraficSourceID, SearchEngineID@1 as SearchEngineID, AdvEngineID@2 as AdvEngineID, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3 as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@4 as URL], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([TraficSourceID@0, SearchEngineID@1, AdvEngineID@2, CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END@3, URL@4], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[TraficSourceID@2 as TraficSourceID, SearchEngineID@3 as SearchEngineID, AdvEngineID@4 as AdvEngineID, CASE WHEN SearchEngineID@3 = 0 AND AdvEngineID@4 = 0 THEN Referer@1 ELSE END as CASE WHEN hits.SearchEngineID = Int64(0) AND hits.AdvEngineID = Int64(0) THEN hits.Referer ELSE Utf8("") END, URL@0 as URL], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@4 = 0, projection=[URL@2, Referer@3, TraficSourceID@5, SearchEngineID@6, AdvEngineID@7] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, URL, Referer, IsRefresh, TraficSourceID, SearchEngineID, AdvEngineID], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND IsRefresh@15 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8, required_guarantees=[CounterID in (62), IsRefresh in (0)] @@ -1105,9 +1105,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@2 DESC], fetch=110 03)----SortExec: TopK(fetch=110), expr=[pageviews@2 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[URLHash@0 as URLHash, EventDate@1 as EventDate, count(Int64(1))@2 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] 08)--------------ProjectionExec: expr=[URLHash@0 as URLHash, CAST(CAST(EventDate@1 AS Int32) AS Date32) as EventDate] 09)----------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[URLHash@5, EventDate@0] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -1135,9 +1135,9 @@ physical_plan 02)--SortPreservingMergeExec: [pageviews@2 DESC], fetch=10010 03)----SortExec: TopK(fetch=10010), expr=[pageviews@2 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight, count(Int64(1))@2 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([WindowClientWidth@0, WindowClientHeight@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[WindowClientWidth@0 as WindowClientWidth, WindowClientHeight@1 as WindowClientHeight], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@1 = 62 AND EventDate@0 >= 15887 AND EventDate@0 <= 15917 AND IsRefresh@2 = 0 AND DontCountHits@5 = 0 AND URLHash@6 = 2868770270353813622, projection=[WindowClientWidth@3, WindowClientHeight@4] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, IsRefresh, WindowClientWidth, WindowClientHeight, DontCountHits, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15887 AND EventDate@5 <= 15917 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0 AND URLHash@103 = 2868770270353813622, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15887 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15917 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11 AND URLHash_null_count@15 != row_count@3 AND URLHash_min@13 <= 2868770270353813622 AND 2868770270353813622 <= URLHash_max@14, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0), URLHash in (2868770270353813622)] @@ -1164,9 +1164,9 @@ physical_plan 02)--SortPreservingMergeExec: [date_trunc(minute, m@0) ASC NULLS LAST], fetch=1010 03)----SortExec: TopK(fetch=1010), expr=[date_trunc(minute, m@0) ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as m, count(Int64(1))@1 as pageviews] -05)--------AggregateExec: mode=FinalPartitioned, gby=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0 as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[date_trunc(minute, to_timestamp_seconds(EventTime@0)) as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[date_trunc(minute, to_timestamp_seconds(EventTime@0)) as date_trunc(Utf8("minute"),to_timestamp_seconds(hits.EventTime))], aggr=[count(Int64(1))] 08)--------------FilterExec: CounterID@2 = 62 AND EventDate@1 >= 15900 AND EventDate@1 <= 15901 AND IsRefresh@3 = 0 AND DontCountHits@4 = 0, projection=[EventTime@0] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, EventDate, CounterID, IsRefresh, DontCountHits], file_type=parquet, predicate=CounterID@6 = 62 AND EventDate@5 >= 15900 AND EventDate@5 <= 15901 AND IsRefresh@15 = 0 AND DontCountHits@61 = 0, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND EventDate_max@4 >= 15900 AND EventDate_null_count@5 != row_count@3 AND EventDate_min@6 <= 15901 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND DontCountHits_null_count@12 != row_count@3 AND DontCountHits_min@10 <= 0 AND 0 <= DontCountHits_max@11, required_guarantees=[CounterID in (62), DontCountHits in (0), IsRefresh in (0)] @@ -1220,9 +1220,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [count(DISTINCT hits.SocialNetwork)@1 DESC], fetch=10 02)--SortExec: TopK(fetch=10), expr=[count(DISTINCT hits.SocialNetwork)@1 DESC], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[BrowserCountry@0 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[BrowserCountry@0 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)] 04)------RepartitionExec: partitioning=Hash([BrowserCountry@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[BrowserCountry@2 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[BrowserCountry@2 as BrowserCountry], aggr=[count(DISTINCT hits.SocialNetwork), count(DISTINCT hits.HitColor), count(DISTINCT hits.BrowserLanguage), count(DISTINCT hits.SocialAction)] 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[HitColor, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction], file_type=parquet query TIIII diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index ffe08b7dd513..a1c0e6303a76 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -48,9 +48,9 @@ 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))], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -64,9 +64,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a, count(Int64(1))@1 as cnt] 02)--FilterExec: count(Int64(1))@1 > 0 -03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] query II diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 1f89f6422ea4..0659b9c208f9 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -97,9 +97,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 05)--------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -06)----------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]], stream=RawPartialHashAggregateStream +06)----------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]] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 2788973465a2..8c055c25caeb 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2021,9 +2021,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 05)--------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -06)----------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]], stream=RawPartialHashAggregateStream +06)----------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]] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)], projection=[col0@2, col1@3, col2@4, col0@0, col1@1] 09)----------------DataSourceExec: partitions=1, partition_sizes=[3] @@ -2987,9 +2987,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 -06)----------AggregateExec: mode=Partial, 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]], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, 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]] 07)------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR @@ -3021,9 +3021,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 -06)----------AggregateExec: mode=Partial, 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]], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, 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]] 07)------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3183,9 +3183,9 @@ 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]], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 07)------------SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3218,9 +3218,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 05)--------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -06)----------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]], stream=RawPartialHashAggregateStream +06)----------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]] 07)------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[true] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3418,9 +3418,9 @@ 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=[], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] 05)--------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[2] @@ -3486,9 +3486,9 @@ 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)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[sum(l.amount)] 05)--------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)] 07)------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1, projection=[amount@1, sn@2, amount@3] 08)--------------DataSourceExec: partitions=1, partition_sizes=[2] 09)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -3632,9 +3632,9 @@ 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=[], stream=PartialFinalHashAggregateStream +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=[] 05)--------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 -06)----------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=[], stream=RawPartialHashAggregateStream +06)----------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=[] 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 08)--------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] 09)----------------BoundedWindowAggExec: wdw=[sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Field { "sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING": nullable Float64 }, frame: ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING], mode=[Sorted] @@ -3859,9 +3859,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 03)----RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 -04)------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]], stream=RawPartialHashAggregateStream +04)------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]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1, maintains_sort_order=true 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true @@ -4226,9 +4226,9 @@ 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)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 03)----RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] query TT @@ -4241,12 +4241,12 @@ 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)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] 03)----RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -04)------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)], stream=RawPartialHashAggregateStream -05)--------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(x@0 AS Float64) as alias1], aggr=[], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(x@0 AS Float64) as alias1], aggr=[] 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. @@ -4440,12 +4440,12 @@ 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)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] 05)--------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)], stream=RawPartialHashAggregateStream -07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] +07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3] 08)--------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 -09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3], stream=RawPartialHashAggregateStream +09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[sum(aggregate_test_100.c3) as alias2, max(aggregate_test_100.c4) as alias3] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt index d789ccac3aed..16ab90651c8b 100644 --- a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt +++ b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt @@ -142,26 +142,26 @@ physical_plan 01)SortPreservingMergeExec: [channel@0 ASC, brand@1 ASC] 02)--SortExec: expr=[channel@0 ASC, brand@1 ASC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[channel@0 as channel, brand@1 as brand, sum(sub.total)@3 as grand_total] -04)------AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, brand@1 as brand, __grouping_id@2 as __grouping_id], aggr=[sum(sub.total)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[channel@0 as channel, brand@1 as brand, __grouping_id@2 as __grouping_id], aggr=[sum(sub.total)] 05)--------RepartitionExec: partitioning=Hash([channel@0, brand@1, __grouping_id@2], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[(NULL as channel, NULL as brand), (channel@0 as channel, NULL as brand), (channel@0 as channel, brand@1 as brand)], aggr=[sum(sub.total)] 07)------------InterleaveExec 08)--------------ProjectionExec: expr=[store as channel, brand@0 as brand, sum(sales.amount)@1 as total] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=PartialFinalHashAggregateStream +09)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 10)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=RawPartialHashAggregateStream +11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 12)----------------------FilterExec: channel@0 = store, projection=[brand@1, amount@2] 13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] 14)--------------ProjectionExec: expr=[web as channel, brand@0 as brand, sum(sales.amount)@1 as total] -15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=PartialFinalHashAggregateStream +15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 16)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 -17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=RawPartialHashAggregateStream +17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 18)----------------------FilterExec: channel@0 = web, projection=[brand@1, amount@2] 19)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] 20)--------------ProjectionExec: expr=[catalog as channel, brand@0 as brand, sum(sales.amount)@1 as total] -21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=PartialFinalHashAggregateStream +21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 22)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 -23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)], stream=RawPartialHashAggregateStream +23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 24)----------------------FilterExec: channel@0 = catalog, projection=[brand@1, amount@2] 25)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index c9b3374bd267..3bf101f203fb 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -481,7 +481,7 @@ datafusion.optimizer.repartition_joins true Should DataFusion repartition data u datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below ```text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ``` would turn into the plan below which performs better in multithreaded environments ```text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ``` datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail -datafusion.optimizer.subset_repartition_threshold 4 Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): ```text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ```, stream=RawPartialHashAggregateStream +datafusion.optimizer.subset_repartition_threshold 4 Partition count threshold for subset satisfaction optimization. When the current partition count is >= this threshold, DataFusion will skip repartitioning if the required partitioning expression is a subset of the current partition expression such as Hash(a) satisfies Hash(a, b). When the current partition count is < this threshold, DataFusion will repartition to increase parallelism even when subset satisfaction applies. Set to 0 to always repartition (disable subset satisfaction optimization). Set to a high value to always use subset satisfaction. Example (subset_repartition_threshold = 4): ```text Hash([a]) satisfies Hash([a, b]) because (Hash([a, b]) is subset of Hash([a]) If current partitions (3) < threshold (4), repartition: AggregateExec: mode=FinalPartitioned, gby=[a, b], aggr=[SUM(x)] RepartitionExec: partitioning=Hash([a, b], 8), input_partitions=3 AggregateExec: mode=Partial, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 3) If current partitions (8) >= threshold (4), use subset satisfaction: AggregateExec: mode=SinglePartitioned, gby=[a, b], aggr=[SUM(x)] DataSourceExec: file_groups={...}, output_partitioning=Hash([a], 8) ``` 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.use_statistics_registry false When set to true, the physical plan optimizer uses the pluggable `StatisticsRegistry` for statistics propagation across operators. This enables more accurate cardinality estimates compared to each operator's built-in `partition_statistics`. datafusion.runtime.file_statistics_cache_limit 20M Maximum memory to use for file statistics cache. Supports suffixes K (kilobytes), M (megabytes), and G (gigabytes) or '0' for 0. Example: '2G' for 2 gigabytes. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 26b062587da7..b9d163d87759 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1422,9 +1422,9 @@ physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[v0@2, v1@3, sum(t1.v1)@0] 02)--CoalescePartitionsExec 03)----ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] -04)------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] 05)--------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] 07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index fce9f7f8f360..e0be63fe7152 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1338,9 +1338,9 @@ 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=[], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[] 02)--RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1393,9 +1393,9 @@ 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))], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1420,9 +1420,9 @@ 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=[], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/lateral_join.slt b/datafusion/sqllogictest/test_files/lateral_join.slt index 5bb11e767466..cae3e6715324 100644 --- a/datafusion/sqllogictest/test_files/lateral_join.slt +++ b/datafusion/sqllogictest/test_files/lateral_join.slt @@ -773,9 +773,9 @@ physical_plan 04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, __always_true@3, cnt@1] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------ProjectionExec: expr=[count(Int64(1))@1 as cnt, t1_id@0 as t1_id, true as __always_true] -07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=PartialFinalHashAggregateStream +07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 08)------------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=1 -09)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +09)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 10)----------------DataSourceExec: partitions=1, partition_sizes=[1] # Verify LEFT lateral without aggregate decorrelates to left join diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 4f98fa4fc90e..ca2b36727d62 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -422,9 +422,9 @@ 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=[], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] 02)--RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=1 -03)----AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] 04)------DataSourceExec: partitions=1 statement ok @@ -652,9 +652,9 @@ 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)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[sum(ordered_table.a)] 05)--------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true @@ -1037,9 +1037,9 @@ physical_plan 03)----SortPreservingMergeExec: [sx@1 DESC], fetch=4 04)------SortExec: TopK(fetch=4), expr=[sx@1 DESC], preserve_partitioning=[true] 05)--------ProjectionExec: expr=[g@0 as g, sum(t22489.x)@1 as sx, sum(t22489.y)@2 as sy] -06)----------AggregateExec: mode=FinalPartitioned, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=FinalPartitioned, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)] 07)------------RepartitionExec: partitioning=Hash([g@0], 4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)], stream=RawPartialHashAggregateStream +08)--------------AggregateExec: mode=Partial, gby=[g@0 as g], aggr=[sum(t22489.x), sum(t22489.y)] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index ccbf179cfaab..6907e489e690 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -274,9 +274,9 @@ physical_plan 01)SortPreservingMergeExec: [c2@0 ASC NULLS LAST] 02)--SortExec: expr=[c2@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] 05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -293,9 +293,9 @@ physical_plan 01)SortPreservingMergeExec: [total_sal@1 ASC NULLS LAST, c2@0 ASC NULLS LAST] 02)--SortExec: expr=[total_sal@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] 05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -312,9 +312,9 @@ physical_plan 01)SortPreservingMergeExec: [c2@0 ASC NULLS LAST, abs(c2@0) ASC NULLS LAST] 02)--SortExec: expr=[c2@0 ASC NULLS LAST, abs(c2@0) ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] 05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true @@ -1034,15 +1034,15 @@ 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=[], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 06)----------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 08)--------------ProjectionExec: expr=[column1@0 as t] 09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------ProjectionExec: expr=[1 as m, t@0 as t] -11)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[], stream=PartialFinalHashAggregateStream +11)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 12)----------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 -13)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[], stream=RawPartialHashAggregateStream +13)------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 14)--------------ProjectionExec: expr=[column1@0 as t] 15)----------------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 41a1021781eb..175d7d90cd8e 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -227,9 +227,9 @@ logical_plan 03)----TableScan: fact_table projection=[value, f_dkey] physical_plan 01)ProjectionExec: expr=[f_dkey@0 as f_dkey, count(Int64(1))@1 as count(*), sum(fact_table.value)@2 as sum(fact_table.value)] -02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet # Verify results without optimization @@ -589,9 +589,9 @@ logical_plan 03)----TableScan: high_cardinality_table projection=[value, f_dkey] physical_plan 01)ProjectionExec: expr=[f_dkey@0 as f_dkey, count(Int64(1))@1 as count(*), sum(high_cardinality_table.value)@2 as sum(high_cardinality_table.value)] -02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=E/data.parquet]]}, projection=[value, f_dkey], file_type=parquet query TIR rowsort @@ -636,9 +636,9 @@ logical_plan 06)------SubqueryAlias: d 07)--------TableScan: dimension_table_partitioned projection=[env, d_dkey] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)] 02)--RepartitionExec: partitioning=Hash([f_dkey@0, env@1], 3), input_partitions=3 -03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)] 04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[value@2, f_dkey@3, env@0] 05)--------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet @@ -681,9 +681,9 @@ logical_plan 06)------SubqueryAlias: d 07)--------TableScan: dimension_table_partitioned projection=[env, d_dkey] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)] 02)--RepartitionExec: partitioning=Hash([f_dkey@0, env@1], 3), input_partitions=3 -03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)] 04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[value@2, f_dkey@3, env@0] 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet 06)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index f3b474067147..344aef1f92cf 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -790,9 +790,9 @@ logical_plan 04)------TableScan: multi_struct projection=[s] physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as multi_struct.s[label], sum(__datafusion_extracted_2)@1 as sum(multi_struct.s[value])] -02)--AggregateExec: mode=FinalPartitioned, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] 03)----RepartitionExec: partitioning=Hash([__datafusion_extracted_1@0], 4), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] 05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet # Verify correctness diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index f2fd1089079c..40bfe79dcc63 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -741,9 +741,9 @@ Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)], projection=[a@0, min_value@2], metrics=[output_rows=2, output_batches=2, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_build.parquet]]}, projection=[a], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=14.45% (64/443)] 03)--ProjectionExec: expr=[a@0 as a, min(join_agg_probe.value)@1 as min_value], metrics=[output_rows=2, output_batches=2] -04)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], stream=PartialFinalHashAggregateStream, metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] +04)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] 05)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1, metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] -06)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], stream=RawPartialHashAggregateStream, metrics=[output_rows=2, output_batches=1, spill_count=0, spilled_rows=0, skipped_aggregation_rows=0, reduction_factor=100% (2/2)] +06)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=1, spill_count=0, spilled_rows=0, skipped_aggregation_rows=0, reduction_factor=100% (2/2)] 07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet]]}, projection=[a, value], file_type=parquet, predicate=DynamicFilter [ a@0 >= h1 AND a@0 <= h2 AND a@0 IN (SET) ([h1, h2]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= h1 AND a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND (a_null_count@1 != row_count@2 AND a_min@3 <= h1 AND h1 <= a_max@0 OR a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND h2 <= a_max@0), required_guarantees=[a in (h1, h2)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=4, predicate_cache_records=2, scan_efficiency_ratio=19.07% (151/792)] statement ok diff --git a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index 2cf9ef52038c..923a51afc8df 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -180,7 +180,7 @@ group by grouping sets ((), (id)) ---- physical_plan 01)ProjectionExec: expr=[max(agg_dyn_test.id)@2 as max(agg_dyn_test.id)] -02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)] 03)----RepartitionExec: partitioning=Hash([id@0, __grouping_id@1], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[(NULL as id), (id@0 as id)], aggr=[max(agg_dyn_test.id)] 05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] @@ -497,9 +497,9 @@ EXPLAIN SELECT a, count(b) AS cnt FROM agg_filter_pushdown GROUP BY a HAVING cou physical_plan 01)ProjectionExec: expr=[a@0 as a, count(agg_filter_pushdown.b)@1 as cnt] 02)--FilterExec: count(agg_filter_pushdown.b)@1 > 5 -03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)], stream=PartialFinalHashAggregateStream +03)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)] 04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)], stream=RawPartialHashAggregateStream +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(agg_filter_pushdown.b)] 06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_filter_pushdown.parquet]]}, projection=[a, b], file_type=parquet statement ok diff --git a/datafusion/sqllogictest/test_files/qualify.slt b/datafusion/sqllogictest/test_files/qualify.slt index e0a68be0f238..68aae16d9014 100644 --- a/datafusion/sqllogictest/test_files/qualify.slt +++ b/datafusion/sqllogictest/test_files/qualify.slt @@ -322,9 +322,9 @@ physical_plan 05)--------SortExec: expr=[dept@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=4 07)------------FilterExec: sum(users.salary)@2 > 20000.00, projection=[dept@0, salary@1] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)], stream=PartialFinalHashAggregateStream +08)--------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept, salary@1 as salary], aggr=[sum(users.salary)] 09)----------------RepartitionExec: partitioning=Hash([dept@0, salary@1], 4), input_partitions=4 -10)------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)], stream=RawPartialHashAggregateStream +10)------------------AggregateExec: mode=Partial, gby=[dept@1 as dept, salary@0 as salary], aggr=[sum(users.salary)] 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)----------------------FilterExec: salary@0 > 5000.00 13)------------------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -353,9 +353,9 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() ORDER BY [sum(users.salary) DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 07)------------SortPreservingMergeExec: [sum(users.salary)@1 DESC] 08)--------------SortExec: expr=[sum(users.salary)@1 DESC], preserve_partitioning=[true] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)], stream=PartialFinalHashAggregateStream +09)----------------AggregateExec: mode=FinalPartitioned, gby=[dept@0 as dept], aggr=[sum(users.salary)] 10)------------------RepartitionExec: partitioning=Hash([dept@0], 4), input_partitions=1 -11)--------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)], stream=RawPartialHashAggregateStream +11)--------------------AggregateExec: mode=Partial, gby=[dept@1 as dept], aggr=[sum(users.salary)] 12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] # Clean up diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 95758642cd65..cf913caefc52 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -43,9 +43,9 @@ 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)], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 -03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # disable round robin repartitioning @@ -59,9 +59,9 @@ 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)], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 -03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index 4f944075b3b0..dbf31dec5e11 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -369,9 +369,9 @@ physical_plan 01)SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] 02)--SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[env@0 as env, time_bin@1 as time_bin, avg(a.max_bin_value)@2 as avg_max_value] -04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)] 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 -06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) 09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3, preserve_order=true, sort_exprs=f_dkey@0 ASC NULLS LAST, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 ASC NULLS LAST @@ -466,9 +466,9 @@ physical_plan 01)SortPreservingMergeExec: [env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST] 02)--SortExec: expr=[env@0 ASC NULLS LAST, time_bin@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[env@0 as env, time_bin@1 as time_bin, avg(a.max_bin_value)@2 as avg_max_value] -04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, time_bin@1 as time_bin], aggr=[avg(a.max_bin_value)] 05)--------RepartitionExec: partitioning=Hash([env@0, time_bin@1], 3), input_partitions=3 -06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) 09)----------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[f_dkey@4, env@0, timestamp@2, value@3] diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index d22be62d4743..762f5c11333d 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1573,9 +1573,9 @@ 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))], stream=PartialFinalHashAggregateStream +02)--AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[c2@0 as c2] 07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index d292f47113c3..dd195b0ff487 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -204,9 +204,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -237,9 +237,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int * Float64(1)) + Int64(1)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int * Float64(1))@1 + 1 as sum(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))] 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int) as sum(t2.t2_int * Float64(1))] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -270,9 +270,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -304,9 +304,9 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------FilterExec: sum(t2.t2_int)@1 < 3, projection=[sum(t2.t2_int)@1, t2_id@0] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: partitions=1, partition_sizes=[2] 10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index 5ff3c2bf2ba0..6df93a3daabf 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -168,9 +168,9 @@ 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]], stream=PartialFinalHashAggregateStream +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]] 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -07)------------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]], stream=RawPartialHashAggregateStream +07)------------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]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index cca3ae69983f..db4c98161c20 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -50,9 +50,9 @@ 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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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))], stream=PartialFinalHashAggregateStream +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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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)--------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 -06)----------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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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))], stream=RawPartialHashAggregateStream +06)----------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(__common_expr_1) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * 1 + lineitem.l_tax) as 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))] 07)------------ProjectionExec: expr=[l_extendedprice@0 * (1 - l_discount@1) as __common_expr_1, l_quantity@2 as l_quantity, l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 08)--------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_extendedprice@1, l_discount@2, l_quantity@0, l_tax@3, l_returnflag@4, l_linestatus@5] 09)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 55b2a4c4ddfd..210468450d45 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -72,9 +72,9 @@ 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 * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream +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 * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] 08)--------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 475ec102ea31..e8a224867df0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -78,9 +78,9 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[value@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] 05)--------FilterExec: CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@1 AS Decimal128(38, 15)) > scalar_subquery() -06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 07)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)], stream=RawPartialHashAggregateStream +08)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] 10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index af212701f014..84a6598cb992 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -62,9 +62,9 @@ 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 = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 05)--------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = 1-URGENT OR orders.o_orderpriority = 2-HIGH THEN 1 ELSE 0 END) as 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 != 1-URGENT AND orders.o_orderpriority != 2-HIGH THEN 1 ELSE 0 END) as sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 08)--------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 2f14ee491f91..24e23e4dbd0a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -56,9 +56,9 @@ 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))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(Int64(1))] 07)------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 08)--------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 09)----------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 64cdc27956f2..5af08fa79c92 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -78,17 +78,17 @@ physical_plan 06)----------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 07)--------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] 08)----------FilterExec: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 = scalar_subquery() -09)------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream +09)------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 10)--------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -11)----------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream +11)----------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 12)------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] 13)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false 14)--AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] 15)----CoalescePartitionsExec 16)------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] 17)--------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -18)----------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream +18)----------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 19)------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -20)--------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream +20)--------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 21)----------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] 22)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 16b8a908c589..0d5e0c030321 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -68,12 +68,12 @@ 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)], stream=PartialFinalHashAggregateStream +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)] 05)--------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 -06)----------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)], stream=RawPartialHashAggregateStream -07)------------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=[], stream=PartialFinalHashAggregateStream +06)----------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)] +07)------------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)--------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 -09)----------------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=[], stream=RawPartialHashAggregateStream +09)----------------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=[] 10)------------------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] 11)--------------------CoalescePartitionsExec 12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 40f29cfbb734..9f375a583f77 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -63,7 +63,7 @@ physical_plan 10)--------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] 11)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:0..597773], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:597773..1195546], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1195546..1793319], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1793319..2391090]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false 12)----------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] -13)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], stream=PartialFinalHashAggregateStream +13)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] 14)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -15)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)], stream=RawPartialHashAggregateStream +15)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] 16)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 7fb5b1fb7846..831072092b25 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -81,7 +81,7 @@ physical_plan 12)----------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 13)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false 14)--------FilterExec: sum(lineitem.l_quantity)@1 > 300.00, projection=[l_orderkey@0] -15)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], stream=PartialFinalHashAggregateStream +15)----------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] 16)------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)], stream=RawPartialHashAggregateStream +17)--------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] 18)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index c26fd7bad2b6..e471c2c23d2e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -125,9 +125,9 @@ physical_plan 24)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false 25)------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 26)--------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -27)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], stream=PartialFinalHashAggregateStream +27)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 28)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -29)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)], stream=RawPartialHashAggregateStream +29)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 30)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] 31)------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 32)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 09f5385d6436..76876160e2bb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -102,8 +102,8 @@ physical_plan 19)----------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] 20)------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:0..597773], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:597773..1195546], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1195546..1793319], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl:1793319..2391090]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false 21)----------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] -22)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], stream=PartialFinalHashAggregateStream +22)------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] 23)--------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -24)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)], stream=RawPartialHashAggregateStream +24)----------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] 25)------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] 26)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index a4da51047fef..5e9192d67753 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -93,9 +93,9 @@ 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))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] 07)------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] 08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 09)----------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 0e598dda40b5..97f017eff226 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -76,9 +76,9 @@ physical_plan 02)--SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] 03)----SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[cntrycode@0 as cntrycode, count(Int64(1))@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] -05)--------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)], stream=PartialFinalHashAggregateStream +05)--------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] 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)], stream=RawPartialHashAggregateStream +07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(Int64(1)), sum(custsale.c_acctbal)] 08)--------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] 10)------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 13443eb4bfd9..0007666f1536 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -56,9 +56,9 @@ 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))], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] 07)------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 08)--------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 09)----------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index c8759b43d658..6cbc9c4bef26 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -70,9 +70,9 @@ 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 * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * 1 - lineitem.l_discount) as sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] 08)--------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 518814605112..4bcb738d621d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -87,9 +87,9 @@ 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)], stream=PartialFinalHashAggregateStream +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)] 05)--------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 -06)----------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)], stream=RawPartialHashAggregateStream +06)----------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)] 07)------------ProjectionExec: expr=[n_name@0 as supp_nation, n_name@1 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@3 * (1 - l_discount@4) as volume] 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@4, n_name@6, l_shipdate@2, l_extendedprice@0, l_discount@1] 09)----------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 704681551f07..189d501ce207 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -93,9 +93,9 @@ 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 = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] 05)--------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = BRAZIL THEN all_nations.volume ELSE 0.0000 END) as sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] 07)------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@0) as o_year, l_extendedprice@1 * (1 - l_discount@2) as volume, n_name@3 as nation] 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[o_orderdate@2, l_extendedprice@0, l_discount@1, n_name@4] 09)----------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 2a1b17b6a5ac..84b8e6fffd16 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -78,9 +78,9 @@ 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)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@2 * (1 - l_discount@3) - ps_supplycost@4 * l_quantity@5 as amount] 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_extendedprice@1, l_discount@2, ps_supplycost@4, l_quantity@0] 09)----------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index e052a8f537e6..a48ede604968 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -233,9 +233,9 @@ logical_plan 05)----Projection: t2.name || Utf8View("_new") AS name 06)------TableScan: t2 projection=[name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 -03)----AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 04)------UnionExec 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------DataSourceExec: partitions=1, partition_sizes=[1] @@ -313,9 +313,9 @@ logical_plan 05)----Filter: t1.id = Int32(2) 06)------TableScan: t1 projection=[id, name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] 02)--RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 05)--------UnionExec 06)----------FilterExec: id@0 = 1 @@ -334,9 +334,9 @@ logical_plan 02)--Filter: t1.id = Int32(1) OR t1.id = Int32(2) 03)----TableScan: t1 projection=[id, name] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream +01)AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] 02)--RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream +03)----AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------FilterExec: id@0 = 1 OR id@0 = 2 06)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -381,18 +381,18 @@ physical_plan 04)------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream +07)----AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] 08)------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -09)--------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream +09)--------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)------------DataSourceExec: partitions=1, partition_sizes=[1] 12)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 13)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], NullsEqual: true 14)------CoalescePartitionsExec 15)--------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -16)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[], stream=PartialFinalHashAggregateStream +16)----------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] 17)------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -18)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[], stream=RawPartialHashAggregateStream +18)--------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 19)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 20)------------------DataSourceExec: partitions=1, partition_sizes=[1] 21)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -444,18 +444,18 @@ physical_plan 01)UnionExec 02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true 03)----CoalescePartitionsExec -04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 05)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------DataSourceExec: partitions=1, partition_sizes=[1] 11)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], NullsEqual: true 12)----CoalescePartitionsExec -13)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream +13)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 14)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -15)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream +15)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 16)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 17)--------------DataSourceExec: partitions=1, partition_sizes=[1] 18)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -513,14 +513,14 @@ 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=[], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 05)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream +06)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[], stream=PartialFinalHashAggregateStream +09)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 10)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -11)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[], stream=RawPartialHashAggregateStream +11)----------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 12)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 13)--------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -570,9 +570,9 @@ physical_plan 06)----------CoalescePartitionsExec 07)------------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1))] 08)--------------ProjectionExec: expr=[] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[], stream=PartialFinalHashAggregateStream +09)----------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] 10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[], stream=RawPartialHashAggregateStream +11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 14)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index f02b22b61ca1..1c614f6a22c1 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)], stream=PartialFinalHashAggregateStream +04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[max(d.a)] 05)--------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 07)------------UnionExec @@ -1877,9 +1877,9 @@ 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=[], stream=PartialFinalHashAggregateStream +06)----------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] 07)------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[], stream=RawPartialHashAggregateStream +08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 09)----------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true From 9ff9f8bb2bc09f83544b022841f756722667fb44 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Tue, 2 Jun 2026 15:32:22 +0800 Subject: [PATCH 8/8] cleanup --- .../src/aggregates/raw_partial_hash.rs | 908 +----------------- .../aggregates/raw_partial_hash/hash_table.rs | 773 +++++++++++++++ .../raw_partial_hash/partial_final.rs | 181 ++++ uv.lock | 20 +- 4 files changed, 974 insertions(+), 908 deletions(-) create mode 100644 datafusion/physical-plan/src/aggregates/raw_partial_hash/hash_table.rs create mode 100644 datafusion/physical-plan/src/aggregates/raw_partial_hash/partial_final.rs diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs index d8f1a8c288aa..a4af1d66cf31 100644 --- a/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash.rs @@ -28,684 +28,28 @@ //! `partial state -> partial state`, and single-stage aggregation on //! `GroupedHashAggregateStream` for now. -use std::collections::HashMap; -use std::marker::PhantomData; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow::array::{ArrayRef, AsArray, BooleanArray, new_null_array}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::{Result, internal_err}; +use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use datafusion_expr::{EmitTo, GroupsAccumulator}; use futures::ready; use futures::stream::{Stream, StreamExt}; -use super::group_values::{ - GroupByMetrics, GroupValues, new_unordered_blocked_group_values, - new_unordered_group_values, -}; -use super::row_hash::{ - create_blocked_group_accumulator, create_blocked_group_accumulators, - create_group_accumulator, -}; -use super::{ - AggregateExec, PhysicalGroupBy, aggregate_expressions, evaluate_group_by, - group_id_array, max_duplicate_ordinal, -}; -use crate::metrics::{ - BaselineMetrics, MetricBuilder, MetricCategory, RecordOutput, SpillMetrics, -}; +use self::hash_table::{AggregateHashTable, ExecutionState, RawPartial}; +use super::{AggregateExec, AggregateMode}; +use crate::metrics::{BaselineMetrics, RecordOutput, SpillMetrics}; use crate::stream::EmptyRecordBatchStream; -use crate::{InputOrderMode, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; +use crate::{InputOrderMode, RecordBatchStream, SendableRecordBatchStream}; -#[derive(Debug, Clone)] -enum ExecutionState { - ReadingInput, - ProducingOutput, - Done, -} - -struct HashAggregateAccumulator { - /// Arguments to pass to this accumulator. - /// - /// Example: `CORR(x, y)` stores two expressions here, while `SUM(x)` stores one. - arguments: Vec>, - - /// Optional `FILTER` expression for this accumulator. - /// - /// Example: `SUM(x) FILTER (WHERE x > 10)` stores the `x > 10` predicate. - filter: Option>, - - /// Accumulator state for all groups for one aggregate expression. - accumulator: Box, -} - -struct EvaluatedHashAggregateAccumulator { - arguments: Vec, - filter: Option, -} - -struct EvaluatedAggregateBatch { - /// One entry per grouping set; each entry contains all evaluated group key - /// arrays for the current input batch. - grouping_set_args: Vec>, - - /// Evaluated arguments and filters, one entry per aggregate expression. - accumulator_args: Vec, -} - -fn can_try_blocked_hash_aggregate(agg: &AggregateExec) -> bool { - !agg.group_by.has_grouping_set() && agg.group_by.groups().len() == 1 -} - -pub(crate) fn can_use_blocked_hash_aggregate(agg: &AggregateExec) -> Result { - if !can_try_blocked_hash_aggregate(agg) { - return Ok(false); - } - - let input_schema = agg.input().schema(); - let group_schema = agg.group_by.group_schema(&input_schema)?; - let Some(group_values) = new_unordered_blocked_group_values(&group_schema, 1)? else { - return Ok(false); - }; - - if !group_values.supports_blocked_emit() { - return Ok(false); - } - - for agg_expr in agg.aggr_expr.iter() { - let Some(accumulator) = create_blocked_group_accumulator(agg_expr, 1)? else { - return Ok(false); - }; - if !accumulator.supports_blocked_emit() { - return Ok(false); - } - } - - Ok(true) -} - -impl HashAggregateAccumulator { - fn new( - arguments: Vec>, - filter: Option>, - accumulator: Box, - ) -> Self { - Self { - arguments, - filter, - accumulator, - } - } - - fn evaluate(&self, batch: &RecordBatch) -> Result { - let arguments = self - .arguments - .iter() - .map(|expr| { - expr.evaluate(batch) - .and_then(|value| value.into_array(batch.num_rows())) - }) - .collect::>()?; - - let filter = self - .filter - .as_ref() - .map(|filter| { - filter - .evaluate(batch) - .and_then(|value| value.into_array(batch.num_rows())) - }) - .transpose()?; - - Ok(EvaluatedHashAggregateAccumulator { arguments, filter }) - } - - fn update_batch( - &mut self, - values: &EvaluatedHashAggregateAccumulator, - group_indices: &[usize], - total_num_groups: usize, - ) -> Result<()> { - let filter = values.filter.as_ref().map(|filter| filter.as_boolean()); - self.accumulator.update_batch( - &values.arguments, - group_indices, - filter, - total_num_groups, - ) - } - - fn merge_batch( - &mut self, - values: &EvaluatedHashAggregateAccumulator, - group_indices: &[usize], - total_num_groups: usize, - ) -> Result<()> { - debug_assert!(values.filter.is_none()); - self.accumulator.merge_batch( - &values.arguments, - group_indices, - None, - total_num_groups, - ) - } - - fn evaluate_final(&mut self, emit_to: EmitTo) -> Result { - self.ensure_emit_supported(emit_to)?; - self.accumulator.evaluate(emit_to) - } - - fn state(&mut self, emit_to: EmitTo) -> Result> { - self.ensure_emit_supported(emit_to)?; - self.accumulator.state(emit_to) - } - - fn ensure_emit_supported(&self, emit_to: EmitTo) -> Result<()> { - if matches!(emit_to, EmitTo::Block) && !self.accumulator.supports_blocked_emit() { - return internal_err!( - "EmitTo::Block is not supported by this GroupsAccumulator" - ); - } - Ok(()) - } - - fn supports_convert_to_state(&self) -> bool { - self.accumulator.supports_convert_to_state() - } - - fn null_arguments(&self, input_schema: &SchemaRef) -> Result> { - self.arguments - .iter() - .map(|expr| { - let data_type = expr.data_type(input_schema)?; - Ok(new_null_array(&data_type, 1)) - }) - .collect() - } -} - -/// Hash table mode that consumes raw input rows and produces partial state. -struct RawPartial; - -/// Hash table mode that consumes partial state and produces final values. -struct PartialFinal; - -/// Hash table state for grouped raw-partial aggregation. -/// -/// This owns the coupled state for: -/// - evaluating group keys, -/// - interning each distinct group, -/// - mapping each input row to its group index, -/// - evaluating aggregate inputs, -/// - updating per-group accumulator state. -struct AggregateHashTable { - /// Grouping and accumulator-specific timing metrics. - group_by_metrics: GroupByMetrics, - - /// Raw input schema, used to evaluate expressions and synthesize empty - /// grouping-set rows. - input_schema: SchemaRef, - - /// Output schema: group columns followed by aggregate state or final values. - output_schema: SchemaRef, - - /// Maximum rows per emitted output batch. - batch_size: usize, - - /// True when group values and all accumulators use the same internal block - /// size and can emit one block per output batch. - blocked_output: bool, - - /// GROUP BY expressions evaluated for each input batch. - group_by: Arc, +mod hash_table; +mod partial_final; - /// Interned group keys. Accumulator state is stored separately by group index. - group_values: Box, - - /// Group index for each row in the current input batch. - /// - /// Each value indexes into `group_values`, and the same index is used by every - /// accumulator to update that group's aggregate state. - batch_group_indices: Vec, - - /// One item per aggregate expression. - /// - /// Example: `COUNT(x), SUM(y)` creates two items. Each item owns the input - /// expressions, optional filter, and accumulator state for all groups. - accumulators: Vec, - - /// Full output built once after input is exhausted. - output_batch: Option, - - /// Offset of the next row to slice from `output_batch`. - output_batch_offset: usize, - - /// True once all output rows have been emitted. - output_finished: bool, - - _mode: PhantomData, -} - -impl AggregateHashTable { - fn new_with_filters( - agg: &AggregateExec, - partition: usize, - output_schema: SchemaRef, - batch_size: usize, - filters: Vec>>, - ) -> Result { - let input_schema = agg.input().schema(); - let aggregate_arguments = aggregate_expressions( - &agg.aggr_expr, - &agg.mode, - agg.group_by.num_group_exprs(), - )?; - - let group_schema = agg.group_by.group_schema(&input_schema)?; - let can_try_blocked_output = can_try_blocked_hash_aggregate(agg); - - let blocked_group_values = if can_try_blocked_output { - new_unordered_blocked_group_values(&group_schema, batch_size)? - } else { - None - }; - let blocked_accumulators = if can_try_blocked_output { - create_blocked_group_accumulators(&agg.aggr_expr, batch_size)? - } else { - None - }; - - let (group_values, accumulator_impls, blocked_output) = - match (blocked_group_values, blocked_accumulators) { - (Some(group_values), Some(accumulators)) - if group_values.supports_blocked_emit() - && accumulators.iter().all(|acc| acc.supports_blocked_emit()) => - { - (group_values, accumulators, true) - } - _ => { - let group_values = new_unordered_group_values(group_schema)?; - let accumulators = agg - .aggr_expr - .iter() - .map(create_group_accumulator) - .collect::>>()?; - (group_values, accumulators, false) - } - }; - - let accumulators: Vec<_> = aggregate_arguments - .into_iter() - .zip(filters) - .zip(accumulator_impls) - .map(|((arguments, filter), accumulator)| { - HashAggregateAccumulator::new(arguments, filter, accumulator) - }) - .collect(); - - Ok(Self { - group_by_metrics: GroupByMetrics::new(&agg.metrics, partition), - input_schema, - output_schema, - batch_size, - blocked_output, - group_by: Arc::clone(&agg.group_by), - group_values, - batch_group_indices: Default::default(), - accumulators, - output_batch: None, - output_batch_offset: 0, - output_finished: false, - _mode: PhantomData, - }) - } - - fn evaluate_batch(&self, batch: &RecordBatch) -> Result { - let timer = self.group_by_metrics.time_calculating_group_ids.timer(); - // outer vec: one per each grouping set - // inner vec: all group by exprs for the current grouping set - let grouping_set_args = evaluate_group_by(&self.group_by, batch)?; - drop(timer); - - let timer = self.group_by_metrics.aggregate_arguments_time.timer(); - // The evaluated args for each accumulator - let accumulator_args = self - .accumulators - .iter() - .map(|acc| acc.evaluate(batch)) - .collect::>>()?; - drop(timer); - - Ok(EvaluatedAggregateBatch { - grouping_set_args, - accumulator_args, - }) - } - - fn next_output_batch_from( - &mut self, - build_output_batch: impl FnOnce(&mut Self) -> Result>, - ) -> Result> { - if self.output_finished { - return Ok(None); - } - - if self.output_batch.is_none() { - self.output_batch = build_output_batch(self)?; - self.output_batch_offset = 0; - } - - let Some(batch) = self.output_batch.as_ref() else { - self.output_finished = true; - return Ok(None); - }; - - debug_assert!(self.batch_size > 0); - let output_len = self - .batch_size - .max(1) - .min(batch.num_rows() - self.output_batch_offset); - let output = batch.slice(self.output_batch_offset, output_len); - self.output_batch_offset += output_len; - - if self.output_batch_offset == batch.num_rows() { - self.output_batch = None; - self.output_batch_offset = 0; - self.output_finished = true; - } - - debug_assert!(output.num_rows() > 0); - debug_assert!(output.num_rows() <= self.batch_size.max(1)); - Ok(Some(output)) - } - - fn next_blocked_output_batch_from( - &mut self, - build_output_batch: impl FnOnce(&mut Self) -> Result>, - ) -> Result> { - debug_assert!(self.blocked_output); - - if self.output_finished { - return Ok(None); - } - - let output = build_output_batch(self)?; - if output.is_none() { - self.output_finished = true; - } - - Ok(output) - } - - fn memory_size(&self) -> usize { - let acc = self - .accumulators - .iter() - .map(|acc| acc.accumulator.size()) - .sum::(); - let output = self - .output_batch - .as_ref() - .map(RecordBatch::get_array_memory_size) - .unwrap_or_default(); - - acc + self.group_values.size() - + self.batch_group_indices.allocated_size() - + output - } - - fn clear(&mut self) { - self.group_values.clear_shrink(0); - self.batch_group_indices.clear(); - self.batch_group_indices.shrink_to(0); - self.output_batch = None; - self.output_batch_offset = 0; - self.output_finished = false; - } -} - -impl AggregateHashTable { - fn new( - agg: &AggregateExec, - partition: usize, - output_schema: SchemaRef, - batch_size: usize, - ) -> Result { - let table = Self::new_with_filters( - agg, - partition, - output_schema, - batch_size, - agg.filter_expr.iter().cloned().collect(), - )?; - - if table - .accumulators - .iter() - .all(|acc| acc.supports_convert_to_state()) - { - let _skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) - .with_category(MetricCategory::Rows) - .counter("skipped_aggregation_rows", partition); - } - - Ok(table) - } - - fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { - let evaluated_batch = self.evaluate_batch(batch)?; - - let timer = self.group_by_metrics.aggregation_time.timer(); - for group_values in &evaluated_batch.grouping_set_args { - self.group_values - .intern(group_values, &mut self.batch_group_indices)?; - let group_indices = &self.batch_group_indices; - let total_num_groups = self.group_values.len(); - - for (acc, values) in self - .accumulators - .iter_mut() - .zip(evaluated_batch.accumulator_args.iter()) - { - acc.update_batch(values, group_indices, total_num_groups)?; - } - } - drop(timer); - - Ok(()) - } - - fn next_output_batch(&mut self) -> Result> { - if self.blocked_output { - self.next_blocked_output_batch_from(Self::build_blocked_output_batch) - } else { - self.next_output_batch_from(Self::build_output_batch) - } - } - - fn build_output_batch(&mut self) -> Result> { - self.init_empty_grouping_sets()?; - - if self.group_values.is_empty() { - return Ok(None); - } - - let timer = self.group_by_metrics.emitting_time.timer(); - let mut output = self.group_values.emit(EmitTo::All)?; - - for acc in self.accumulators.iter_mut() { - output.extend(acc.state(EmitTo::All)?); - } - - let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; - debug_assert!(batch.num_rows() > 0); - drop(timer); - Ok(Some(batch)) - } - - fn build_blocked_output_batch(&mut self) -> Result> { - self.init_empty_grouping_sets()?; - - if self.group_values.is_empty() { - return Ok(None); - } - - let timer = self.group_by_metrics.emitting_time.timer(); - let mut output = self.group_values.emit(EmitTo::Block)?; - - for acc in self.accumulators.iter_mut() { - output.extend(acc.state(EmitTo::Block)?); - } - - let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; - debug_assert!(batch.num_rows() > 0); - debug_assert!(batch.num_rows() <= self.batch_size); - drop(timer); - Ok(Some(batch)) - } - - fn init_empty_grouping_sets(&mut self) -> Result<()> { - if !self.group_by.has_grouping_set() || !self.group_values.is_empty() { - return Ok(()); - } - - let max_ordinal = max_duplicate_ordinal(self.group_by.groups()); - let mut ordinals: HashMap<&[bool], usize> = HashMap::new(); - let group_schema = self.group_by.group_schema(&self.input_schema)?; - let n_expr = self.group_by.expr().len(); - let mut any_interned = false; - - for group in self.group_by.groups() { - let ordinal = { - let entry = ordinals.entry(group.as_slice()).or_insert(0); - let ordinal = *entry; - *entry += 1; - ordinal - }; - - if !group.iter().all(|&is_null| is_null) { - continue; - } - - let mut cols: Vec = group_schema - .fields() - .iter() - .take(n_expr) - .map(|field| new_null_array(field.data_type(), 1)) - .collect(); - cols.push(group_id_array(group, ordinal, max_ordinal, 1)?); - - self.group_values - .intern(&cols, &mut self.batch_group_indices)?; - any_interned = true; - } - - if any_interned { - let total_groups = self.group_values.len(); - let false_filter = BooleanArray::from(vec![false]); - for acc in self.accumulators.iter_mut() { - let null_args = acc.null_arguments(&self.input_schema)?; - let values = EvaluatedHashAggregateAccumulator { - arguments: null_args, - filter: Some(Arc::new(false_filter.clone())), - }; - acc.update_batch(&values, &[0], total_groups)?; - } - } - - Ok(()) - } -} - -impl AggregateHashTable { - fn new( - agg: &AggregateExec, - partition: usize, - output_schema: SchemaRef, - batch_size: usize, - ) -> Result { - Self::new_with_filters( - agg, - partition, - output_schema, - batch_size, - vec![None; agg.aggr_expr.len()], - ) - } - - fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { - let evaluated_batch = self.evaluate_batch(batch)?; - - let timer = self.group_by_metrics.aggregation_time.timer(); - for group_values in &evaluated_batch.grouping_set_args { - self.group_values - .intern(group_values, &mut self.batch_group_indices)?; - let group_indices = &self.batch_group_indices; - let total_num_groups = self.group_values.len(); - - for (acc, values) in self - .accumulators - .iter_mut() - .zip(evaluated_batch.accumulator_args.iter()) - { - acc.merge_batch(values, group_indices, total_num_groups)?; - } - } - drop(timer); - - Ok(()) - } - - fn next_output_batch(&mut self) -> Result> { - if self.blocked_output { - self.next_blocked_output_batch_from(Self::build_blocked_output_batch) - } else { - self.next_output_batch_from(Self::build_output_batch) - } - } - - fn build_output_batch(&mut self) -> Result> { - if self.group_values.is_empty() { - return Ok(None); - } - - let timer = self.group_by_metrics.emitting_time.timer(); - let mut output = self.group_values.emit(EmitTo::All)?; - - for acc in self.accumulators.iter_mut() { - output.push(acc.evaluate_final(EmitTo::All)?); - } - - let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; - debug_assert!(batch.num_rows() > 0); - drop(timer); - Ok(Some(batch)) - } - - fn build_blocked_output_batch(&mut self) -> Result> { - if self.group_values.is_empty() { - return Ok(None); - } - - let timer = self.group_by_metrics.emitting_time.timer(); - let mut output = self.group_values.emit(EmitTo::Block)?; - - for acc in self.accumulators.iter_mut() { - output.push(acc.evaluate_final(EmitTo::Block)?); - } - - let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; - debug_assert!(batch.num_rows() > 0); - debug_assert!(batch.num_rows() <= self.batch_size); - drop(timer); - Ok(Some(batch)) - } -} +pub(crate) use hash_table::can_use_blocked_hash_aggregate; +pub(crate) use partial_final::PartialFinalHashAggregateStream; /// Hash aggregate stream for grouped `AggregateMode::Partial`. /// @@ -756,7 +100,7 @@ impl RawPartialHashAggregateStream { context: &Arc, partition: usize, ) -> Result { - debug_assert_eq!(agg.mode, super::AggregateMode::Partial); + debug_assert_eq!(agg.mode, AggregateMode::Partial); debug_assert_eq!(agg.input_order_mode, InputOrderMode::Linear); let schema = Arc::clone(&agg.schema); @@ -872,235 +216,3 @@ impl RecordBatchStream for RawPartialHashAggregateStream { Arc::clone(&self.schema) } } - -/// `AggregateMode::FinalPartitioned`. -/// -/// Input: partial state, such as `sum(x), count(x)` for `avg(x)`. -/// Output: final values, such as `avg(x)`. -pub(crate) struct PartialFinalHashAggregateStream { - /// Output schema: group columns followed by final aggregate value columns. - schema: SchemaRef, - - /// Input batches containing partial aggregate state rows. - input: SendableRecordBatchStream, - - /// Controls whether the stream is reading input, emitting output, or done. - exec_state: ExecutionState, - - /// Hash table and accumulator state for all groups seen so far. - hash_table: AggregateHashTable, - - /// Execution metrics shared with the aggregate plan node. - baseline_metrics: BaselineMetrics, - - /// Memory reservation for group keys and accumulators. - reservation: MemoryReservation, -} - -impl PartialFinalHashAggregateStream { - pub fn new( - agg: &AggregateExec, - context: &Arc, - partition: usize, - ) -> Result { - debug_assert!(matches!( - agg.mode, - super::AggregateMode::Final | super::AggregateMode::FinalPartitioned - )); - debug_assert_eq!(agg.input_order_mode, InputOrderMode::Linear); - - let schema = Arc::clone(&agg.schema); - let input = agg.input.execute(partition, Arc::clone(context))?; - let batch_size = context.session_config().batch_size(); - let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); - - // Preserve the existing aggregate metric surface for this plan node. - let _spill_metrics = SpillMetrics::new(&agg.metrics, partition); - - let hash_table = AggregateHashTable::::new( - agg, - partition, - Arc::clone(&schema), - batch_size, - )?; - - let reservation = - MemoryConsumer::new(format!("PartialFinalHashAggregateStream[{partition}]")) - .register(context.memory_pool()); - - Ok(Self { - schema, - input, - exec_state: ExecutionState::ReadingInput, - hash_table, - baseline_metrics, - reservation, - }) - } -} - -impl Stream for PartialFinalHashAggregateStream { - type Item = Result; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); - - loop { - match &self.exec_state { - ExecutionState::ReadingInput => { - match ready!(self.input.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - let timer = elapsed_compute.timer(); - let result = self.hash_table.aggregate_batch(&batch); - timer.done(); - - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); - } - - if let Err(e) = - self.reservation.try_resize(self.hash_table.memory_size()) - { - return Poll::Ready(Some(Err(e))); - } - } - Some(Err(e)) => { - return Poll::Ready(Some(Err(e))); - } - None => { - let input_schema = self.input.schema(); - self.input = - Box::pin(EmptyRecordBatchStream::new(input_schema)); - - self.exec_state = ExecutionState::ProducingOutput; - } - } - } - - ExecutionState::ProducingOutput => { - let timer = elapsed_compute.timer(); - let result = self.hash_table.next_output_batch(); - timer.done(); - - match result { - Ok(Some(batch)) => { - let _ = self - .reservation - .try_resize(self.hash_table.memory_size()); - debug_assert!(batch.num_rows() > 0); - return Poll::Ready(Some(Ok( - batch.record_output(&self.baseline_metrics) - ))); - } - Ok(None) => { - let _ = self - .reservation - .try_resize(self.hash_table.memory_size()); - self.exec_state = ExecutionState::Done; - } - Err(e) => return Poll::Ready(Some(Err(e))), - } - } - - ExecutionState::Done => { - self.hash_table.clear(); - let _ = self.reservation.try_resize(self.hash_table.memory_size()); - return Poll::Ready(None); - } - } - } - } -} - -impl RecordBatchStream for PartialFinalHashAggregateStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::Int64Array; - use arrow::array::types::Int64Type; - use arrow::datatypes::{DataType, Field, Schema}; - - #[test] - fn blocked_group_values_emit_blocks() -> Result<()> { - let schema = Arc::new(Schema::new(vec![Field::new( - "user_id", - DataType::Int64, - false, - )])); - let mut group_values = - new_unordered_blocked_group_values(&schema, 3)?.expect("blocked Int64"); - let values: ArrayRef = - Arc::new(Int64Array::from(vec![10, 20, 30, 40, 50, 60, 70])); - let mut groups = vec![]; - - group_values.intern(&[values], &mut groups)?; - - assert_eq!(groups, vec![0, 1, 2, 3, 4, 5, 6]); - assert_eq!(group_values.len(), 7); - - let block = group_values.emit(EmitTo::Block)?; - assert_eq!( - block[0].as_primitive::().values().as_ref(), - &[10, 20, 30] - ); - - let block = group_values.emit(EmitTo::Block)?; - assert_eq!( - block[0].as_primitive::().values().as_ref(), - &[40, 50, 60] - ); - - let block = group_values.emit(EmitTo::Block)?; - assert_eq!( - block[0].as_primitive::().values().as_ref(), - &[70] - ); - assert!(group_values.is_empty()); - - Ok(()) - } - - #[test] - fn blocked_group_values_support_primitive_keys() -> Result<()> { - let schema = Arc::new(Schema::new(vec![Field::new( - "user_id", - DataType::UInt32, - false, - )])); - let group_values = - new_unordered_blocked_group_values(&schema, 3)?.expect("blocked UInt32"); - - assert!(group_values.supports_blocked_emit()); - - Ok(()) - } - - #[test] - fn non_blocked_group_values_reject_block_emit() -> Result<()> { - let schema = Arc::new(Schema::new(vec![Field::new( - "user_id", - DataType::Int64, - false, - )])); - let mut group_values = new_unordered_group_values(schema)?; - - let error = group_values - .emit(EmitTo::Block) - .expect_err("non-blocked group values should reject EmitTo::Block"); - - assert!(matches!( - error, - datafusion_common::DataFusionError::Internal(_) - )); - - Ok(()) - } -} diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash/hash_table.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash/hash_table.rs new file mode 100644 index 000000000000..80543710ce3d --- /dev/null +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash/hash_table.rs @@ -0,0 +1,773 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashMap; +use std::marker::PhantomData; +use std::sync::Arc; + +use arrow::array::{ArrayRef, AsArray, BooleanArray, new_null_array}; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::{Result, internal_err}; +use datafusion_execution::memory_pool::proxy::VecAllocExt; +use datafusion_expr::{EmitTo, GroupsAccumulator}; + +use crate::PhysicalExpr; +use crate::aggregates::group_values::{ + GroupByMetrics, GroupValues, new_unordered_blocked_group_values, + new_unordered_group_values, +}; +use crate::aggregates::row_hash::{ + create_blocked_group_accumulator, create_blocked_group_accumulators, + create_group_accumulator, +}; +use crate::aggregates::{ + AggregateExec, PhysicalGroupBy, aggregate_expressions, evaluate_group_by, + group_id_array, max_duplicate_ordinal, +}; +use crate::metrics::{MetricBuilder, MetricCategory}; + +#[derive(Debug, Clone)] +pub(super) enum ExecutionState { + ReadingInput, + ProducingOutput, + Done, +} + +struct HashAggregateAccumulator { + /// Arguments to pass to this accumulator. + /// + /// Example: `CORR(x, y)` stores two expressions here, while `SUM(x)` stores one. + arguments: Vec>, + + /// Optional `FILTER` expression for this accumulator. + /// + /// Example: `SUM(x) FILTER (WHERE x > 10)` stores the `x > 10` predicate. + filter: Option>, + + /// Accumulator state for all groups for one aggregate expression. + accumulator: Box, +} + +struct EvaluatedHashAggregateAccumulator { + arguments: Vec, + filter: Option, +} + +struct EvaluatedAggregateBatch { + /// One entry per grouping set; each entry contains all evaluated group key + /// arrays for the current input batch. + grouping_set_args: Vec>, + + /// Evaluated arguments and filters, one entry per aggregate expression. + accumulator_args: Vec, +} + +fn can_try_blocked_hash_aggregate(agg: &AggregateExec) -> bool { + !agg.aggr_expr.is_empty() + && !agg.group_by.has_grouping_set() + && agg.group_by.groups().len() == 1 +} + +pub(crate) fn can_use_blocked_hash_aggregate(agg: &AggregateExec) -> Result { + if !can_try_blocked_hash_aggregate(agg) { + return Ok(false); + } + + let input_schema = agg.input().schema(); + let group_schema = agg.group_by.group_schema(&input_schema)?; + let Some(group_values) = new_unordered_blocked_group_values(&group_schema, 1)? else { + return Ok(false); + }; + + if !group_values.supports_blocked_emit() { + return Ok(false); + } + + for agg_expr in agg.aggr_expr.iter() { + let Some(accumulator) = create_blocked_group_accumulator(agg_expr, 1)? else { + return Ok(false); + }; + if !accumulator.supports_blocked_emit() { + return Ok(false); + } + } + + Ok(true) +} + +impl HashAggregateAccumulator { + fn new( + arguments: Vec>, + filter: Option>, + accumulator: Box, + ) -> Self { + Self { + arguments, + filter, + accumulator, + } + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + let arguments = self + .arguments + .iter() + .map(|expr| { + expr.evaluate(batch) + .and_then(|value| value.into_array(batch.num_rows())) + }) + .collect::>()?; + + let filter = self + .filter + .as_ref() + .map(|filter| { + filter + .evaluate(batch) + .and_then(|value| value.into_array(batch.num_rows())) + }) + .transpose()?; + + Ok(EvaluatedHashAggregateAccumulator { arguments, filter }) + } + + fn update_batch( + &mut self, + values: &EvaluatedHashAggregateAccumulator, + group_indices: &[usize], + total_num_groups: usize, + ) -> Result<()> { + let filter = values.filter.as_ref().map(|filter| filter.as_boolean()); + self.accumulator.update_batch( + &values.arguments, + group_indices, + filter, + total_num_groups, + ) + } + + fn merge_batch( + &mut self, + values: &EvaluatedHashAggregateAccumulator, + group_indices: &[usize], + total_num_groups: usize, + ) -> Result<()> { + debug_assert!(values.filter.is_none()); + self.accumulator.merge_batch( + &values.arguments, + group_indices, + None, + total_num_groups, + ) + } + + fn evaluate_final(&mut self, emit_to: EmitTo) -> Result { + self.ensure_emit_supported(emit_to)?; + self.accumulator.evaluate(emit_to) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + self.ensure_emit_supported(emit_to)?; + self.accumulator.state(emit_to) + } + + fn ensure_emit_supported(&self, emit_to: EmitTo) -> Result<()> { + if matches!(emit_to, EmitTo::Block) && !self.accumulator.supports_blocked_emit() { + return internal_err!( + "EmitTo::Block is not supported by this GroupsAccumulator" + ); + } + Ok(()) + } + + fn supports_convert_to_state(&self) -> bool { + self.accumulator.supports_convert_to_state() + } + + fn null_arguments(&self, input_schema: &SchemaRef) -> Result> { + self.arguments + .iter() + .map(|expr| { + let data_type = expr.data_type(input_schema)?; + Ok(new_null_array(&data_type, 1)) + }) + .collect() + } +} + +/// Hash table mode that consumes raw input rows and produces partial state. +pub(super) struct RawPartial; + +/// Hash table mode that consumes partial state and produces final values. +pub(super) struct PartialFinal; + +/// Hash table state for grouped raw-partial and partial-final aggregation. +/// +/// This owns the coupled state for: +/// - evaluating group keys, +/// - interning each distinct group, +/// - mapping each input row to its group index, +/// - evaluating aggregate inputs, +/// - updating per-group accumulator state. +pub(super) struct AggregateHashTable { + /// Grouping and accumulator-specific timing metrics. + group_by_metrics: GroupByMetrics, + + /// Raw input schema, used to evaluate expressions and synthesize empty + /// grouping-set rows. + input_schema: SchemaRef, + + /// Output schema: group columns followed by aggregate state or final values. + output_schema: SchemaRef, + + /// Maximum rows per emitted output batch. + batch_size: usize, + + /// True when group values and all accumulators use the same internal block + /// size and can emit one block per output batch. + blocked_output: bool, + + /// GROUP BY expressions evaluated for each input batch. + group_by: Arc, + + /// Interned group keys. Accumulator state is stored separately by group index. + group_values: Box, + + /// Group index for each row in the current input batch. + /// + /// Each value indexes into `group_values`, and the same index is used by every + /// accumulator to update that group's aggregate state. + batch_group_indices: Vec, + + /// One item per aggregate expression. + /// + /// Example: `COUNT(x), SUM(y)` creates two items. Each item owns the input + /// expressions, optional filter, and accumulator state for all groups. + accumulators: Vec, + + /// Full output built once after input is exhausted. + output_batch: Option, + + /// Offset of the next row to slice from `output_batch`. + output_batch_offset: usize, + + /// True once all output rows have been emitted. + output_finished: bool, + + _mode: PhantomData, +} + +impl AggregateHashTable { + fn new_with_filters( + agg: &AggregateExec, + partition: usize, + output_schema: SchemaRef, + batch_size: usize, + filters: Vec>>, + ) -> Result { + let input_schema = agg.input().schema(); + let aggregate_arguments = aggregate_expressions( + &agg.aggr_expr, + &agg.mode, + agg.group_by.num_group_exprs(), + )?; + + let group_schema = agg.group_by.group_schema(&input_schema)?; + let can_try_blocked_output = can_try_blocked_hash_aggregate(agg); + + let blocked_group_values = if can_try_blocked_output { + new_unordered_blocked_group_values(&group_schema, batch_size)? + } else { + None + }; + let blocked_accumulators = if can_try_blocked_output { + create_blocked_group_accumulators(&agg.aggr_expr, batch_size)? + } else { + None + }; + + let (group_values, accumulator_impls, blocked_output) = + match (blocked_group_values, blocked_accumulators) { + (Some(group_values), Some(accumulators)) + if group_values.supports_blocked_emit() + && accumulators.iter().all(|acc| acc.supports_blocked_emit()) => + { + (group_values, accumulators, true) + } + _ => { + let group_values = new_unordered_group_values(group_schema)?; + let accumulators = agg + .aggr_expr + .iter() + .map(create_group_accumulator) + .collect::>>()?; + (group_values, accumulators, false) + } + }; + + let accumulators: Vec<_> = aggregate_arguments + .into_iter() + .zip(filters) + .zip(accumulator_impls) + .map(|((arguments, filter), accumulator)| { + HashAggregateAccumulator::new(arguments, filter, accumulator) + }) + .collect(); + + Ok(Self { + group_by_metrics: GroupByMetrics::new(&agg.metrics, partition), + input_schema, + output_schema, + batch_size, + blocked_output, + group_by: Arc::clone(&agg.group_by), + group_values, + batch_group_indices: Default::default(), + accumulators, + output_batch: None, + output_batch_offset: 0, + output_finished: false, + _mode: PhantomData, + }) + } + + fn evaluate_batch(&self, batch: &RecordBatch) -> Result { + let timer = self.group_by_metrics.time_calculating_group_ids.timer(); + // outer vec: one per each grouping set + // inner vec: all group by exprs for the current grouping set + let grouping_set_args = evaluate_group_by(&self.group_by, batch)?; + drop(timer); + + let timer = self.group_by_metrics.aggregate_arguments_time.timer(); + // The evaluated args for each accumulator + let accumulator_args = self + .accumulators + .iter() + .map(|acc| acc.evaluate(batch)) + .collect::>>()?; + drop(timer); + + Ok(EvaluatedAggregateBatch { + grouping_set_args, + accumulator_args, + }) + } + + fn next_output_batch_from( + &mut self, + build_output_batch: impl FnOnce(&mut Self) -> Result>, + ) -> Result> { + if self.output_finished { + return Ok(None); + } + + if self.output_batch.is_none() { + self.output_batch = build_output_batch(self)?; + self.output_batch_offset = 0; + } + + let Some(batch) = self.output_batch.as_ref() else { + self.output_finished = true; + return Ok(None); + }; + + debug_assert!(self.batch_size > 0); + let output_len = self + .batch_size + .max(1) + .min(batch.num_rows() - self.output_batch_offset); + let output = batch.slice(self.output_batch_offset, output_len); + self.output_batch_offset += output_len; + + if self.output_batch_offset == batch.num_rows() { + self.output_batch = None; + self.output_batch_offset = 0; + self.output_finished = true; + } + + debug_assert!(output.num_rows() > 0); + debug_assert!(output.num_rows() <= self.batch_size.max(1)); + Ok(Some(output)) + } + + fn next_blocked_output_batch_from( + &mut self, + build_output_batch: impl FnOnce(&mut Self) -> Result>, + ) -> Result> { + debug_assert!(self.blocked_output); + + if self.output_finished { + return Ok(None); + } + + let output = build_output_batch(self)?; + if output.is_none() { + self.output_finished = true; + } + + Ok(output) + } + + pub(super) fn memory_size(&self) -> usize { + let acc = self + .accumulators + .iter() + .map(|acc| acc.accumulator.size()) + .sum::(); + let output = self + .output_batch + .as_ref() + .map(RecordBatch::get_array_memory_size) + .unwrap_or_default(); + + acc + self.group_values.size() + + self.batch_group_indices.allocated_size() + + output + } + + pub(super) fn clear(&mut self) { + self.group_values.clear_shrink(0); + self.batch_group_indices.clear(); + self.batch_group_indices.shrink_to(0); + self.output_batch = None; + self.output_batch_offset = 0; + self.output_finished = false; + } +} + +impl AggregateHashTable { + pub(super) fn new( + agg: &AggregateExec, + partition: usize, + output_schema: SchemaRef, + batch_size: usize, + ) -> Result { + let table = Self::new_with_filters( + agg, + partition, + output_schema, + batch_size, + agg.filter_expr.iter().cloned().collect(), + )?; + + if table + .accumulators + .iter() + .all(|acc| acc.supports_convert_to_state()) + { + let _skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) + .with_category(MetricCategory::Rows) + .counter("skipped_aggregation_rows", partition); + } + + Ok(table) + } + + pub(super) fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { + let evaluated_batch = self.evaluate_batch(batch)?; + + let timer = self.group_by_metrics.aggregation_time.timer(); + for group_values in &evaluated_batch.grouping_set_args { + self.group_values + .intern(group_values, &mut self.batch_group_indices)?; + let group_indices = &self.batch_group_indices; + let total_num_groups = self.group_values.len(); + + for (acc, values) in self + .accumulators + .iter_mut() + .zip(evaluated_batch.accumulator_args.iter()) + { + acc.update_batch(values, group_indices, total_num_groups)?; + } + } + drop(timer); + + Ok(()) + } + + pub(super) fn next_output_batch(&mut self) -> Result> { + if self.blocked_output { + self.next_blocked_output_batch_from(Self::build_blocked_output_batch) + } else { + self.next_output_batch_from(Self::build_output_batch) + } + } + + fn build_output_batch(&mut self) -> Result> { + self.init_empty_grouping_sets()?; + + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::All)?; + + for acc in self.accumulators.iter_mut() { + output.extend(acc.state(EmitTo::All)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + drop(timer); + Ok(Some(batch)) + } + + fn build_blocked_output_batch(&mut self) -> Result> { + self.init_empty_grouping_sets()?; + + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::Block)?; + + for acc in self.accumulators.iter_mut() { + output.extend(acc.state(EmitTo::Block)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + debug_assert!(batch.num_rows() <= self.batch_size); + drop(timer); + Ok(Some(batch)) + } + + fn init_empty_grouping_sets(&mut self) -> Result<()> { + if !self.group_by.has_grouping_set() || !self.group_values.is_empty() { + return Ok(()); + } + + let max_ordinal = max_duplicate_ordinal(self.group_by.groups()); + let mut ordinals: HashMap<&[bool], usize> = HashMap::new(); + let group_schema = self.group_by.group_schema(&self.input_schema)?; + let n_expr = self.group_by.expr().len(); + let mut any_interned = false; + + for group in self.group_by.groups() { + let ordinal = { + let entry = ordinals.entry(group.as_slice()).or_insert(0); + let ordinal = *entry; + *entry += 1; + ordinal + }; + + if !group.iter().all(|&is_null| is_null) { + continue; + } + + let mut cols: Vec = group_schema + .fields() + .iter() + .take(n_expr) + .map(|field| new_null_array(field.data_type(), 1)) + .collect(); + cols.push(group_id_array(group, ordinal, max_ordinal, 1)?); + + self.group_values + .intern(&cols, &mut self.batch_group_indices)?; + any_interned = true; + } + + if any_interned { + let total_groups = self.group_values.len(); + let false_filter = BooleanArray::from(vec![false]); + for acc in self.accumulators.iter_mut() { + let null_args = acc.null_arguments(&self.input_schema)?; + let values = EvaluatedHashAggregateAccumulator { + arguments: null_args, + filter: Some(Arc::new(false_filter.clone())), + }; + acc.update_batch(&values, &[0], total_groups)?; + } + } + + Ok(()) + } +} + +impl AggregateHashTable { + pub(super) fn new( + agg: &AggregateExec, + partition: usize, + output_schema: SchemaRef, + batch_size: usize, + ) -> Result { + Self::new_with_filters( + agg, + partition, + output_schema, + batch_size, + vec![None; agg.aggr_expr.len()], + ) + } + + pub(super) fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { + let evaluated_batch = self.evaluate_batch(batch)?; + + let timer = self.group_by_metrics.aggregation_time.timer(); + for group_values in &evaluated_batch.grouping_set_args { + self.group_values + .intern(group_values, &mut self.batch_group_indices)?; + let group_indices = &self.batch_group_indices; + let total_num_groups = self.group_values.len(); + + for (acc, values) in self + .accumulators + .iter_mut() + .zip(evaluated_batch.accumulator_args.iter()) + { + acc.merge_batch(values, group_indices, total_num_groups)?; + } + } + drop(timer); + + Ok(()) + } + + pub(super) fn next_output_batch(&mut self) -> Result> { + if self.blocked_output { + self.next_blocked_output_batch_from(Self::build_blocked_output_batch) + } else { + self.next_output_batch_from(Self::build_output_batch) + } + } + + fn build_output_batch(&mut self) -> Result> { + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::All)?; + + for acc in self.accumulators.iter_mut() { + output.push(acc.evaluate_final(EmitTo::All)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + drop(timer); + Ok(Some(batch)) + } + + fn build_blocked_output_batch(&mut self) -> Result> { + if self.group_values.is_empty() { + return Ok(None); + } + + let timer = self.group_by_metrics.emitting_time.timer(); + let mut output = self.group_values.emit(EmitTo::Block)?; + + for acc in self.accumulators.iter_mut() { + output.push(acc.evaluate_final(EmitTo::Block)?); + } + + let batch = RecordBatch::try_new(Arc::clone(&self.output_schema), output)?; + debug_assert!(batch.num_rows() > 0); + debug_assert!(batch.num_rows() <= self.batch_size); + drop(timer); + Ok(Some(batch)) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::Int64Array; + use arrow::array::types::Int64Type; + use arrow::datatypes::{DataType, Field, Schema}; + + #[test] + fn blocked_group_values_emit_blocks() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "user_id", + DataType::Int64, + false, + )])); + let mut group_values = + new_unordered_blocked_group_values(&schema, 3)?.expect("blocked Int64"); + let values: ArrayRef = + Arc::new(Int64Array::from(vec![10, 20, 30, 40, 50, 60, 70])); + let mut groups = vec![]; + + group_values.intern(&[values], &mut groups)?; + + assert_eq!(groups, vec![0, 1, 2, 3, 4, 5, 6]); + assert_eq!(group_values.len(), 7); + + let block = group_values.emit(EmitTo::Block)?; + assert_eq!( + block[0].as_primitive::().values().as_ref(), + &[10, 20, 30] + ); + + let block = group_values.emit(EmitTo::Block)?; + assert_eq!( + block[0].as_primitive::().values().as_ref(), + &[40, 50, 60] + ); + + let block = group_values.emit(EmitTo::Block)?; + assert_eq!( + block[0].as_primitive::().values().as_ref(), + &[70] + ); + assert!(group_values.is_empty()); + + Ok(()) + } + + #[test] + fn blocked_group_values_support_primitive_keys() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "user_id", + DataType::UInt32, + false, + )])); + let group_values = + new_unordered_blocked_group_values(&schema, 3)?.expect("blocked UInt32"); + + assert!(group_values.supports_blocked_emit()); + + Ok(()) + } + + #[test] + fn non_blocked_group_values_reject_block_emit() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "user_id", + DataType::Int64, + false, + )])); + let mut group_values = new_unordered_group_values(schema)?; + + let error = group_values + .emit(EmitTo::Block) + .expect_err("non-blocked group values should reject EmitTo::Block"); + + assert!(matches!( + error, + datafusion_common::DataFusionError::Internal(_) + )); + + Ok(()) + } +} diff --git a/datafusion/physical-plan/src/aggregates/raw_partial_hash/partial_final.rs b/datafusion/physical-plan/src/aggregates/raw_partial_hash/partial_final.rs new file mode 100644 index 000000000000..4a13b53013e7 --- /dev/null +++ b/datafusion/physical-plan/src/aggregates/raw_partial_hash/partial_final.rs @@ -0,0 +1,181 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; +use std::task::{Context, Poll}; + +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::Result; +use datafusion_execution::TaskContext; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use futures::ready; +use futures::stream::{Stream, StreamExt}; + +use super::hash_table::{AggregateHashTable, ExecutionState, PartialFinal}; +use crate::aggregates::{AggregateExec, AggregateMode}; +use crate::metrics::{BaselineMetrics, RecordOutput, SpillMetrics}; +use crate::stream::EmptyRecordBatchStream; +use crate::{InputOrderMode, RecordBatchStream, SendableRecordBatchStream}; + +/// `AggregateMode::FinalPartitioned`. +/// +/// Input: partial state, such as `sum(x), count(x)` for `avg(x)`. +/// Output: final values, such as `avg(x)`. +pub(crate) struct PartialFinalHashAggregateStream { + /// Output schema: group columns followed by final aggregate value columns. + schema: SchemaRef, + + /// Input batches containing partial aggregate state rows. + input: SendableRecordBatchStream, + + /// Controls whether the stream is reading input, emitting output, or done. + exec_state: ExecutionState, + + /// Hash table and accumulator state for all groups seen so far. + hash_table: AggregateHashTable, + + /// Execution metrics shared with the aggregate plan node. + baseline_metrics: BaselineMetrics, + + /// Memory reservation for group keys and accumulators. + reservation: MemoryReservation, +} + +impl PartialFinalHashAggregateStream { + pub fn new( + agg: &AggregateExec, + context: &Arc, + partition: usize, + ) -> Result { + debug_assert!(matches!( + agg.mode, + AggregateMode::Final | AggregateMode::FinalPartitioned + )); + debug_assert_eq!(agg.input_order_mode, InputOrderMode::Linear); + + let schema = Arc::clone(&agg.schema); + let input = agg.input.execute(partition, Arc::clone(context))?; + let batch_size = context.session_config().batch_size(); + let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); + + // Preserve the existing aggregate metric surface for this plan node. + let _spill_metrics = SpillMetrics::new(&agg.metrics, partition); + + let hash_table = AggregateHashTable::::new( + agg, + partition, + Arc::clone(&schema), + batch_size, + )?; + + let reservation = + MemoryConsumer::new(format!("PartialFinalHashAggregateStream[{partition}]")) + .register(context.memory_pool()); + + Ok(Self { + schema, + input, + exec_state: ExecutionState::ReadingInput, + hash_table, + baseline_metrics, + reservation, + }) + } +} + +impl Stream for PartialFinalHashAggregateStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + + loop { + match &self.exec_state { + ExecutionState::ReadingInput => { + match ready!(self.input.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + let timer = elapsed_compute.timer(); + let result = self.hash_table.aggregate_batch(&batch); + timer.done(); + + if let Err(e) = result { + return Poll::Ready(Some(Err(e))); + } + + if let Err(e) = + self.reservation.try_resize(self.hash_table.memory_size()) + { + return Poll::Ready(Some(Err(e))); + } + } + Some(Err(e)) => { + return Poll::Ready(Some(Err(e))); + } + None => { + let input_schema = self.input.schema(); + self.input = + Box::pin(EmptyRecordBatchStream::new(input_schema)); + + self.exec_state = ExecutionState::ProducingOutput; + } + } + } + + ExecutionState::ProducingOutput => { + let timer = elapsed_compute.timer(); + let result = self.hash_table.next_output_batch(); + timer.done(); + + match result { + Ok(Some(batch)) => { + let _ = self + .reservation + .try_resize(self.hash_table.memory_size()); + debug_assert!(batch.num_rows() > 0); + return Poll::Ready(Some(Ok( + batch.record_output(&self.baseline_metrics) + ))); + } + Ok(None) => { + let _ = self + .reservation + .try_resize(self.hash_table.memory_size()); + self.exec_state = ExecutionState::Done; + } + Err(e) => return Poll::Ready(Some(Err(e))), + } + } + + ExecutionState::Done => { + self.hash_table.clear(); + let _ = self.reservation.try_resize(self.hash_table.memory_size()); + return Poll::Ready(None); + } + } + } + } +} + +impl RecordBatchStream for PartialFinalHashAggregateStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} diff --git a/uv.lock b/uv.lock index f86b732dfd6d..749e0c47c4f8 100644 --- a/uv.lock +++ b/uv.lock @@ -350,7 +350,7 @@ dependencies = [ requires-dist = [ { name = "jinja2", specifier = ">=3.1.6,<4" }, { name = "maturin", specifier = ">=1.13.3,<2" }, - { name = "myst-parser", specifier = ">=5,<6" }, + { name = "myst-parser", specifier = ">=5.1.0,<6" }, { name = "pydata-sphinx-theme", specifier = ">=0.17.1,<1" }, { name = "setuptools", specifier = ">=82.0.1,<83" }, { name = "sphinx", specifier = ">=9,<10" }, @@ -465,14 +465,14 @@ wheels = [ [[package]] name = "markdown-it-py" -version = "4.0.0" +version = "4.2.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "mdurl" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/5b/f5/4ec618ed16cc4f8fb3b701563655a69816155e79e24a17b651541804721d/markdown_it_py-4.0.0.tar.gz", hash = "sha256:cb0a2b4aa34f932c007117b194e945bd74e0ec24133ceb5bac59009cda1cb9f3", size = 73070, upload-time = "2025-08-11T12:57:52.854Z" } +sdist = { url = "https://files.pythonhosted.org/packages/06/ff/7841249c247aa650a76b9ee4bbaeae59370dc8bfd2f6c01f3630c35eb134/markdown_it_py-4.2.0.tar.gz", hash = "sha256:04a21681d6fbb623de53f6f364d352309d4094dd4194040a10fd51833e418d49", size = 82454, upload-time = "2026-05-07T12:08:28.36Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/94/54/e7d793b573f298e1c9013b8c4dade17d481164aa517d1d7148619c2cedbf/markdown_it_py-4.0.0-py3-none-any.whl", hash = "sha256:87327c59b172c5011896038353a81343b6754500a08cd7a4973bb48c6d578147", size = 87321, upload-time = "2025-08-11T12:57:51.923Z" }, + { url = "https://files.pythonhosted.org/packages/b3/81/4da04ced5a082363ecfa159c010d200ecbd959ae410c10c0264a38cac0f5/markdown_it_py-4.2.0-py3-none-any.whl", hash = "sha256:9f7ebbcd14fe59494226453aed97c1070d83f8d24b6fc3a3bcf9a38092641c4a", size = 91687, upload-time = "2026-05-07T12:08:27.182Z" }, ] [[package]] @@ -572,14 +572,14 @@ wheels = [ [[package]] name = "mdit-py-plugins" -version = "0.5.0" +version = "0.6.1" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "markdown-it-py" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b2/fd/a756d36c0bfba5f6e39a1cdbdbfdd448dc02692467d83816dff4592a1ebc/mdit_py_plugins-0.5.0.tar.gz", hash = "sha256:f4918cb50119f50446560513a8e311d574ff6aaed72606ddae6d35716fe809c6", size = 44655, upload-time = "2025-08-11T07:25:49.083Z" } +sdist = { url = "https://files.pythonhosted.org/packages/59/fc/f8d0863f8862f25602c0404d75568e89fb6b4109804645e5cdfb1be5cf56/mdit_py_plugins-0.6.1.tar.gz", hash = "sha256:a2bca0f039f39dbd35fb74ae1b5f998608c437463371f0ff7f49a19a17a114d0", size = 56114, upload-time = "2026-05-13T09:03:38.91Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/fb/86/dd6e5db36df29e76c7a7699123569a4a18c1623ce68d826ed96c62643cae/mdit_py_plugins-0.5.0-py3-none-any.whl", hash = "sha256:07a08422fc1936a5d26d146759e9155ea466e842f5ab2f7d2266dd084c8dab1f", size = 57205, upload-time = "2025-08-11T07:25:47.597Z" }, + { url = "https://files.pythonhosted.org/packages/a5/69/6da5581c6a7fede7dc261bf4e67d6adca4196f176b43288b55b3db395b6e/mdit_py_plugins-0.6.1-py3-none-any.whl", hash = "sha256:214c82fb2ac524472ab6a5bcab1de80f73b50443e187f401bfd77efbc7c6481d", size = 66663, upload-time = "2026-05-13T09:03:37.76Z" }, ] [[package]] @@ -593,7 +593,7 @@ wheels = [ [[package]] name = "myst-parser" -version = "5.0.0" +version = "5.1.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "docutils" }, @@ -604,9 +604,9 @@ dependencies = [ { name = "sphinx", version = "9.0.4", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.12'" }, { name = "sphinx", version = "9.1.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.12'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/33/fa/7b45eef11b7971f0beb29d27b7bfe0d747d063aa29e170d9edd004733c8a/myst_parser-5.0.0.tar.gz", hash = "sha256:f6f231452c56e8baa662cc352c548158f6a16fcbd6e3800fc594978002b94f3a", size = 98535, upload-time = "2026-01-15T09:08:18.036Z" } +sdist = { url = "https://files.pythonhosted.org/packages/21/dc/603751677fff302f34396e206b610f556a59d7fe58b9a2145f54e96b48e8/myst_parser-5.1.0.tar.gz", hash = "sha256:ab69322dc6719dcc7f296479dbb70181b66df6ed315064f92dbc85c0e1bf2f02", size = 101182, upload-time = "2026-05-13T09:38:19.361Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d3/ac/686789b9145413f1a61878c407210e41bfdb097976864e0913078b24098c/myst_parser-5.0.0-py3-none-any.whl", hash = "sha256:ab31e516024918296e169139072b81592336f2fef55b8986aa31c9f04b5f7211", size = 84533, upload-time = "2026-01-15T09:08:16.788Z" }, + { url = "https://files.pythonhosted.org/packages/09/dc/f3dfb7488b770f3f67e6545085bf2abea5172e88f57b8ad25ef860ca704c/myst_parser-5.1.0-py3-none-any.whl", hash = "sha256:9c91c52b3cdb4d94a6506e4fab4e2f296c7623a0da0dcbe6de1565c3dad67a8a", size = 85817, upload-time = "2026-05-13T09:38:17.904Z" }, ] [[package]]