From 9c840b0d83e74f1b01f730754caea7653ae1a876 Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Sat, 21 Sep 2024 10:41:33 +0200 Subject: [PATCH] Fix grouping sets behavior when data contains nulls --- datafusion/core/src/physical_planner.rs | 16 +- .../src/combine_partial_final_agg.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 360 +++++++++++------- .../physical-plan/src/aggregates/row_hash.rs | 11 +- .../sqllogictest/test_files/aggregate.slt | 14 +- 5 files changed, 256 insertions(+), 147 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 84d285fc2509..f41a57452bc3 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -707,10 +707,6 @@ impl DefaultPhysicalPlanner { physical_input_schema.clone(), )?); - // update group column indices based on partial aggregate plan evaluation - let final_group: Vec> = - initial_aggr.output_group_expr(); - let can_repartition = !groups.is_empty() && session_state.config().target_partitions() > 1 && session_state.config().repartition_aggregations(); @@ -731,13 +727,7 @@ impl DefaultPhysicalPlanner { AggregateMode::Final }; - let final_grouping_set = PhysicalGroupBy::new_single( - final_group - .iter() - .enumerate() - .map(|(i, expr)| (expr.clone(), groups.expr()[i].1.clone())) - .collect(), - ); + let final_grouping_set = initial_aggr.group_expr().as_final(); Arc::new(AggregateExec::try_new( next_partition_mode, @@ -2061,7 +2051,7 @@ mod tests { &session_state, ); - let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[false, false, false], [true, false, false], [false, true, false], [false, false, true], [true, true, false], [true, false, true], [false, true, true], [true, true, true]] })"#; + let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[false, false, false], [true, false, false], [false, true, false], [false, false, true], [true, true, false], [true, false, true], [false, true, true], [true, true, true]], num_output_exprs: 4 })"#; assert_eq!(format!("{cube:?}"), expected); @@ -2088,7 +2078,7 @@ mod tests { &session_state, ); - let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[true, true, true], [false, true, true], [false, false, true], [false, false, false]] })"#; + let expected = r#"Ok(PhysicalGroupBy { expr: [(Column { name: "c1", index: 0 }, "c1"), (Column { name: "c2", index: 1 }, "c2"), (Column { name: "c3", index: 2 }, "c3")], null_expr: [(Literal { value: Utf8(NULL) }, "c1"), (Literal { value: Int64(NULL) }, "c2"), (Literal { value: Int64(NULL) }, "c3")], groups: [[true, true, true], [false, true, true], [false, false, true], [false, false, false]], num_output_exprs: 4 })"#; assert_eq!(format!("{rollup:?}"), expected); diff --git a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs index 67e40c9b507e..e5e0ce07c1ff 100644 --- a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs +++ b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs @@ -135,7 +135,7 @@ fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { // Compare output expressions of the partial, and input expressions of the final operator. physical_exprs_equal( - &input_group_by.output_exprs(), + &input_group_by.output_exprs(&AggregateMode::Partial), &final_group_by.input_exprs(), ) && input_group_by.groups() == final_group_by.groups() && input_group_by.null_expr().len() == final_group_by.null_expr().len() diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c3bc7b042e65..945e8079e452 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -35,6 +35,8 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use arrow_array::{UInt16Array, UInt32Array, UInt64Array, UInt8Array}; +use arrow_schema::DataType; use datafusion_common::stats::Precision; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_execution::TaskContext; @@ -108,6 +110,8 @@ impl AggregateMode { } } +const INTERNAL_GROUPING_ID: &str = "grouping_id"; + /// Represents `GROUP BY` clause in the plan (including the more general GROUPING SET) /// In the case of a simple `GROUP BY a, b` clause, this will contain the expression [a, b] /// and a single group [false, false]. @@ -137,6 +141,10 @@ pub struct PhysicalGroupBy { /// expression in null_expr. If `groups[i][j]` is true, then the /// j-th expression in the i-th group is NULL, otherwise it is `expr[j]`. groups: Vec>, + // The number of internal expressions that are used to implement grouping + // sets. These output are removed from the final output and not in `expr` + // as they are generated based on the value in `groups` + num_internal_exprs: usize, } impl PhysicalGroupBy { @@ -146,10 +154,12 @@ impl PhysicalGroupBy { null_expr: Vec<(Arc, String)>, groups: Vec>, ) -> Self { + let num_internal_exprs = if !null_expr.is_empty() { 1 } else { 0 }; Self { expr, null_expr, groups, + num_internal_exprs, } } @@ -161,6 +171,7 @@ impl PhysicalGroupBy { expr, null_expr: vec![], groups: vec![vec![false; num_exprs]], + num_internal_exprs: 0, } } @@ -210,13 +221,114 @@ impl PhysicalGroupBy { .collect() } + /// The number of expressions in the output schema. + fn num_output_exprs(&self, mode: &AggregateMode) -> usize { + let mut num_exprs = self.expr.len(); + if !self.is_single() { + num_exprs += self.num_internal_exprs; + } + if *mode != AggregateMode::Partial { + num_exprs -= self.num_internal_exprs; + } + num_exprs + } + /// Return grouping expressions as they occur in the output schema. - pub fn output_exprs(&self) -> Vec> { - self.expr - .iter() - .enumerate() - .map(|(index, (_, name))| Arc::new(Column::new(name, index)) as _) - .collect() + pub fn output_exprs(&self, mode: &AggregateMode) -> Vec> { + let num_output_exprs = self.num_output_exprs(mode); + let mut output_exprs = Vec::with_capacity(num_output_exprs); + output_exprs.extend( + self.expr + .iter() + .enumerate() + .take(num_output_exprs) + .map(|(index, (_, name))| Arc::new(Column::new(name, index)) as _), + ); + if !self.is_single() && *mode == AggregateMode::Partial { + output_exprs + .push(Arc::new(Column::new(INTERNAL_GROUPING_ID, self.expr.len())) as _); + } + output_exprs + } + + /// Returns the number expression as grouping keys. + fn num_group_exprs(&self) -> usize { + if self.is_single() { + self.expr.len() + } else { + self.expr.len() + self.num_internal_exprs + } + } + + /// Returns the data type of the grouping id. + fn grouping_id_type(&self) -> DataType { + if self.expr.len() <= 8 { + DataType::UInt8 + } else if self.expr.len() <= 16 { + DataType::UInt16 + } else if self.expr.len() <= 32 { + DataType::UInt32 + } else { + DataType::UInt64 + } + } + + /// Returns the fields that are used as the grouping keys. + fn group_fields(&self, input_schema: &Schema) -> Result> { + let mut fields = Vec::with_capacity(self.num_group_exprs()); + for ((expr, name), group_expr_nullable) in + self.expr.iter().zip(self.exprs_nullable().into_iter()) + { + fields.push(Field::new( + name, + expr.data_type(input_schema)?, + group_expr_nullable || expr.nullable(input_schema)?, + )) + } + if !self.is_single() { + fields.push(Field::new( + INTERNAL_GROUPING_ID, + self.grouping_id_type(), + false, + )); + } + Ok(fields) + } + + /// Returns the output fields of the group by. + /// + /// This might be different from the `group_fields` that might contain internal expressions that + /// should not be part of the output schema. + fn output_fields( + &self, + input_schema: &Schema, + mode: &AggregateMode, + ) -> Result> { + let mut fields = self.group_fields(input_schema)?; + fields.truncate(self.num_output_exprs(mode)); + Ok(fields) + } + + /// Returns the `PhysicalGroupBy` for a final aggregation if `self` is used for a partial + /// aggregation. + pub fn as_final(&self) -> PhysicalGroupBy { + let expr: Vec<_> = self + .output_exprs(&AggregateMode::Partial) + .into_iter() + .zip( + self.expr + .iter() + .map(|t| t.1.clone()) + .chain(std::iter::once(INTERNAL_GROUPING_ID.to_owned())), + ) + .collect(); + let num_exprs = expr.len(); + Self { + expr, + null_expr: vec![], + groups: vec![vec![false; num_exprs]], + num_internal_exprs: self.num_internal_exprs, + } } } @@ -320,13 +432,7 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, ) -> Result { - let schema = create_schema( - &input.schema(), - &group_by.expr, - &aggr_expr, - group_by.exprs_nullable(), - mode, - )?; + let schema = create_schema(&input.schema(), &group_by, &aggr_expr, mode)?; let schema = Arc::new(schema); AggregateExec::try_new_with_schema( @@ -456,7 +562,7 @@ impl AggregateExec { /// Grouping expressions as they occur in the output schema pub fn output_group_expr(&self) -> Vec> { - self.group_by.output_exprs() + self.group_by.output_exprs(&AggregateMode::Partial) } /// Aggregate expressions @@ -786,22 +892,13 @@ impl ExecutionPlan for AggregateExec { fn create_schema( input_schema: &Schema, - group_expr: &[(Arc, String)], + group_by: &PhysicalGroupBy, aggr_expr: &[AggregateFunctionExpr], - group_expr_nullable: Vec, mode: AggregateMode, ) -> Result { - let mut fields = Vec::with_capacity(group_expr.len() + aggr_expr.len()); - for (index, (expr, name)) in group_expr.iter().enumerate() { - fields.push(Field::new( - name, - expr.data_type(input_schema)?, - // In cases where we have multiple grouping sets, we will use NULL expressions in - // order to align the grouping sets. So the field must be nullable even if the underlying - // schema field is not. - group_expr_nullable[index] || expr.nullable(input_schema)?, - )) - } + let mut fields = + Vec::with_capacity(group_by.num_output_exprs(&mode) + aggr_expr.len()); + fields.extend(group_by.output_fields(input_schema, &mode)?); match mode { AggregateMode::Partial => { @@ -824,9 +921,8 @@ fn create_schema( Ok(Schema::new(fields)) } -fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { - let group_fields = schema.fields()[0..group_count].to_vec(); - Arc::new(Schema::new(group_fields)) +fn group_schema(input_schema: &Schema, group_by: &PhysicalGroupBy) -> Result { + Ok(Arc::new(Schema::new(group_by.group_fields(input_schema)?))) } /// Determines the lexical ordering requirement for an aggregate expression. @@ -1133,6 +1229,27 @@ fn evaluate_optional( .collect() } +fn group_id_array(group: &[bool], batch: &RecordBatch) -> Result { + if group.len() > 64 { + return not_impl_err!( + "Grouping sets with more than 64 columns are not supported" + ); + } + let group_id = group.iter().fold(0u64, |acc, &is_null| { + (acc << 1) | if is_null { 1 } else { 0 } + }); + let num_rows = batch.num_rows(); + if group.len() <= 8 { + return Ok(Arc::new(UInt8Array::from(vec![group_id as u8; num_rows]))); + } else if group.len() <= 16 { + return Ok(Arc::new(UInt16Array::from(vec![group_id as u16; num_rows]))); + } else if group.len() <= 32 { + return Ok(Arc::new(UInt32Array::from(vec![group_id as u32; num_rows]))); + } else { + return Ok(Arc::new(UInt64Array::from(vec![group_id; num_rows]))); + } +} + /// Evaluate a group by expression against a `RecordBatch` /// /// Arguments: @@ -1165,23 +1282,24 @@ pub(crate) fn evaluate_group_by( }) .collect::>>()?; - Ok(group_by + group_by .groups .iter() .map(|group| { - group - .iter() - .enumerate() - .map(|(idx, is_null)| { - if *is_null { - Arc::clone(&null_exprs[idx]) - } else { - Arc::clone(&exprs[idx]) - } - }) - .collect() + let mut group_values = Vec::with_capacity(group_by.num_group_exprs()); + group_values.extend(group.iter().enumerate().map(|(idx, is_null)| { + if *is_null { + Arc::clone(&null_exprs[idx]) + } else { + Arc::clone(&exprs[idx]) + } + })); + if !group_by.is_single() { + group_values.push(group_id_array(group, batch)?); + } + Ok(group_values) }) - .collect()) + .collect() } #[cfg(test)] @@ -1336,21 +1454,21 @@ mod tests { ) -> Result<()> { let input_schema = input.schema(); - let grouping_set = PhysicalGroupBy { - expr: vec![ + let grouping_set = PhysicalGroupBy::new( + vec![ (col("a", &input_schema)?, "a".to_string()), (col("b", &input_schema)?, "b".to_string()), ], - null_expr: vec![ + vec![ (lit(ScalarValue::UInt32(None)), "a".to_string()), (lit(ScalarValue::Float64(None)), "b".to_string()), ], - groups: vec![ + vec![ vec![false, true], // (a, NULL) vec![true, false], // (NULL, b) vec![false, false], // (a,b) ], - }; + ); let aggregates = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1i8)]) .schema(Arc::clone(&input_schema)) @@ -1377,63 +1495,56 @@ mod tests { let expected = if spill { vec![ - "+---+-----+-----------------+", - "| a | b | COUNT(1)[count] |", - "+---+-----+-----------------+", - "| | 1.0 | 1 |", - "| | 1.0 | 1 |", - "| | 2.0 | 1 |", - "| | 2.0 | 1 |", - "| | 3.0 | 1 |", - "| | 3.0 | 1 |", - "| | 4.0 | 1 |", - "| | 4.0 | 1 |", - "| 2 | | 1 |", - "| 2 | | 1 |", - "| 2 | 1.0 | 1 |", - "| 2 | 1.0 | 1 |", - "| 3 | | 1 |", - "| 3 | | 2 |", - "| 3 | 2.0 | 2 |", - "| 3 | 3.0 | 1 |", - "| 4 | | 1 |", - "| 4 | | 2 |", - "| 4 | 3.0 | 1 |", - "| 4 | 4.0 | 2 |", - "+---+-----+-----------------+", + "+---+-----+-------------+-----------------+", + "| a | b | grouping_id | COUNT(1)[count] |", + "+---+-----+-------------+-----------------+", + "| | 1.0 | 2 | 1 |", + "| | 1.0 | 2 | 1 |", + "| | 2.0 | 2 | 1 |", + "| | 2.0 | 2 | 1 |", + "| | 3.0 | 2 | 1 |", + "| | 3.0 | 2 | 1 |", + "| | 4.0 | 2 | 1 |", + "| | 4.0 | 2 | 1 |", + "| 2 | | 1 | 1 |", + "| 2 | | 1 | 1 |", + "| 2 | 1.0 | 0 | 1 |", + "| 2 | 1.0 | 0 | 1 |", + "| 3 | | 1 | 1 |", + "| 3 | | 1 | 2 |", + "| 3 | 2.0 | 0 | 2 |", + "| 3 | 3.0 | 0 | 1 |", + "| 4 | | 1 | 1 |", + "| 4 | | 1 | 2 |", + "| 4 | 3.0 | 0 | 1 |", + "| 4 | 4.0 | 0 | 2 |", + "+---+-----+-------------+-----------------+", ] } else { vec![ - "+---+-----+-----------------+", - "| a | b | COUNT(1)[count] |", - "+---+-----+-----------------+", - "| | 1.0 | 2 |", - "| | 2.0 | 2 |", - "| | 3.0 | 2 |", - "| | 4.0 | 2 |", - "| 2 | | 2 |", - "| 2 | 1.0 | 2 |", - "| 3 | | 3 |", - "| 3 | 2.0 | 2 |", - "| 3 | 3.0 | 1 |", - "| 4 | | 3 |", - "| 4 | 3.0 | 1 |", - "| 4 | 4.0 | 2 |", - "+---+-----+-----------------+", + "+---+-----+-------------+-----------------+", + "| a | b | grouping_id | COUNT(1)[count] |", + "+---+-----+-------------+-----------------+", + "| | 1.0 | 2 | 2 |", + "| | 2.0 | 2 | 2 |", + "| | 3.0 | 2 | 2 |", + "| | 4.0 | 2 | 2 |", + "| 2 | | 1 | 2 |", + "| 2 | 1.0 | 0 | 2 |", + "| 3 | | 1 | 3 |", + "| 3 | 2.0 | 0 | 2 |", + "| 3 | 3.0 | 0 | 1 |", + "| 4 | | 1 | 3 |", + "| 4 | 3.0 | 0 | 1 |", + "| 4 | 4.0 | 0 | 2 |", + "+---+-----+-------------+-----------------+", ] }; assert_batches_sorted_eq!(expected, &result); - let groups = partial_aggregate.group_expr().expr().to_vec(); - let merge = Arc::new(CoalescePartitionsExec::new(partial_aggregate)); - let final_group: Vec<(Arc, String)> = groups - .iter() - .map(|(_expr, name)| Ok((col(name, &input_schema)?, name.clone()))) - .collect::>()?; - - let final_grouping_set = PhysicalGroupBy::new_single(final_group); + let final_grouping_set = grouping_set.as_final(); let task_ctx = if spill { new_spill_ctx(4, 3160) @@ -1488,11 +1599,11 @@ mod tests { async fn check_aggregates(input: Arc, spill: bool) -> Result<()> { let input_schema = input.schema(); - let grouping_set = PhysicalGroupBy { - expr: vec![(col("a", &input_schema)?, "a".to_string())], - null_expr: vec![], - groups: vec![vec![false]], - }; + let grouping_set = PhysicalGroupBy::new( + vec![(col("a", &input_schema)?, "a".to_string())], + vec![], + vec![vec![false]], + ); let aggregates: Vec = vec![ @@ -1548,13 +1659,7 @@ mod tests { let merge = Arc::new(CoalescePartitionsExec::new(partial_aggregate)); - let final_group: Vec<(Arc, String)> = grouping_set - .expr - .iter() - .map(|(_expr, name)| Ok((col(name, &input_schema)?, name.clone()))) - .collect::>()?; - - let final_grouping_set = PhysicalGroupBy::new_single(final_group); + let final_grouping_set = grouping_set.as_final(); let merged_aggregate = Arc::new(AggregateExec::try_new( AggregateMode::Final, @@ -1810,11 +1915,11 @@ mod tests { let task_ctx = Arc::new(task_ctx); let groups_none = PhysicalGroupBy::default(); - let groups_some = PhysicalGroupBy { - expr: vec![(col("a", &input_schema)?, "a".to_string())], - null_expr: vec![], - groups: vec![vec![false]], - }; + let groups_some = PhysicalGroupBy::new( + vec![(col("a", &input_schema)?, "a".to_string())], + vec![], + vec![vec![false]], + ); // something that allocates within the aggregator let aggregates_v0: Vec = @@ -2291,7 +2396,7 @@ mod tests { )?); let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, + AggregateMode::Single, groups, aggregates.clone(), vec![None], @@ -2303,13 +2408,13 @@ mod tests { collect(aggregate_exec.execute(0, Arc::new(TaskContext::default()))?).await?; let expected = [ - "+-----+-----+-------+----------+", - "| a | b | const | 1[count] |", - "+-----+-----+-------+----------+", - "| | 0.0 | | 32768 |", - "| 0.0 | | | 32768 |", - "| | | 1 | 32768 |", - "+-----+-----+-------+----------+", + "+-----+-----+-------+-------+", + "| a | b | const | 1 |", + "+-----+-----+-------+-------+", + "| | 0.0 | | 32768 |", + "| 0.0 | | | 32768 |", + "| | | 1 | 32768 |", + "+-----+-----+-------+-------+", ]; assert_batches_sorted_eq!(expected, &output); @@ -2502,25 +2607,24 @@ mod tests { .build()?, ]; - let grouping_set = PhysicalGroupBy { - expr: vec![ + let grouping_set = PhysicalGroupBy::new( + vec![ (col("a", &input_schema)?, "a".to_string()), (col("b", &input_schema)?, "b".to_string()), ], - null_expr: vec![ + vec![ (lit(ScalarValue::Float32(None)), "a".to_string()), (lit(ScalarValue::Float32(None)), "b".to_string()), ], - groups: vec![ + vec![ vec![false, true], // (a, NULL) vec![false, false], // (a,b) ], - }; + ); let aggr_schema = create_schema( &input_schema, - &grouping_set.expr, + &grouping_set, &aggr_expr, - grouping_set.exprs_nullable(), AggregateMode::Final, )?; let expected_schema = Schema::new(vec![ diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 60efc7711216..614fae1c5091 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -456,13 +456,13 @@ impl GroupedHashAggregateStream { let aggregate_arguments = aggregates::aggregate_expressions( &agg.aggr_expr, &agg.mode, - agg_group_by.expr.len(), + agg_group_by.num_group_exprs(), )?; // arguments for aggregating spilled data is the same as the one for final aggregation let merging_aggregate_arguments = aggregates::aggregate_expressions( &agg.aggr_expr, &AggregateMode::Final, - agg_group_by.expr.len(), + agg_group_by.num_group_exprs(), )?; let filter_expressions = match agg.mode { @@ -480,7 +480,7 @@ impl GroupedHashAggregateStream { .map(create_group_accumulator) .collect::>()?; - let group_schema = group_schema(&agg_schema, agg_group_by.expr.len()); + let group_schema = group_schema(&agg.input().schema(), &agg_group_by)?; let spill_expr = group_schema .fields .into_iter() @@ -498,7 +498,7 @@ impl GroupedHashAggregateStream { let (ordering, _) = agg .properties() .equivalence_properties() - .find_longest_permutation(&agg_group_by.output_exprs()); + .find_longest_permutation(&agg_group_by.output_exprs(&agg.mode)); let group_ordering = GroupOrdering::try_new( &group_schema, &agg.input_order_mode, @@ -851,6 +851,9 @@ impl GroupedHashAggregateStream { } let mut output = self.group_values.emit(emit_to)?; + if !spilling { + output.truncate(self.group_by.num_output_exprs(&self.mode)); + } if let EmitTo::First(n) = emit_to { self.group_ordering.remove_groups(n); } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 576abe5c6f5a..c146d730806f 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3512,6 +3512,18 @@ SELECT MIN(value), MAX(value) FROM integers_with_nulls ---- 1 5 +# grouping_sets with null values +query II rowsort +SELECT value, min(value) FROM integers_with_nulls GROUP BY CUBE(value) +---- +1 1 +3 3 +4 4 +5 5 +NULL 1 +NULL NULL + + statement ok DROP TABLE integers_with_nulls; @@ -4876,7 +4888,7 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c2, c3] physical_plan 01)GlobalLimitExec: skip=0, fetch=3 -02)--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[3] +02)--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3, grouping_id@2 as grouping_id], aggr=[], lim=[3] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[(NULL as c2, NULL as c3), (c2@0 as c2, NULL as c3), (c2@0 as c2, c3@1 as c3)], aggr=[] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1