From 07910bbc728ab889cbc4968785f5496c4b22320b Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Sat, 27 Jul 2024 16:29:27 +0000 Subject: [PATCH] Adding type coercion --- .../expr/src/type_coercion/aggregates.rs | 27 +- .../expr/src/type_coercion/functions.rs | 1 + datafusion/functions-aggregate/src/min_max.rs | 177 +- .../sqllogictest/test_files/aggregate.slt | 5520 ----------------- .../test_files/aggregate_debug.slt | 28 + 5 files changed, 215 insertions(+), 5538 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/aggregate.slt create mode 100644 datafusion/sqllogictest/test_files/aggregate_debug.slt diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 2fc6ac7f0d4c4..fba2e9e2dbf91 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -16,11 +16,11 @@ // under the License. use crate::TypeSignature; - use arrow::datatypes::{ DataType, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION, DECIMAL256_MAX_SCALE, }; +use std::ops::Deref; use datafusion_common::{internal_err, plan_err, Result}; @@ -142,6 +142,22 @@ pub fn check_arg_count( Ok(()) } +pub fn get_min_max_result_type(input_types: &[DataType]) -> Result> { + // make sure that the input types only has one element. + assert_eq!(input_types.len(), 1); + // min and max support the dictionary data type + // unpack the dictionary to get the value + match &input_types[0] { + DataType::Dictionary(_, dict_value_type) => { + // TODO add checker, if the value type is complex data type + Ok(vec![dict_value_type.deref().clone()]) + } + // TODO add checker for datatype which min and max supported + // For example, the `Struct` and `Map` type are not supported in the MIN and MAX function + _ => Ok(input_types.to_vec()), + } +} + /// function return type of a sum pub fn sum_return_type(arg_type: &DataType) -> Result { match arg_type { @@ -311,6 +327,15 @@ pub fn coerce_avg_type(func_name: &str, arg_types: &[DataType]) -> Result Result<()> { + let data_type = + DataType::Dictionary(Box::new(DataType::Utf8), Box::new((DataType::Int32))); + let result = get_min_max_result_type(&[data_type])?; + assert_eq!(result, vec![DataType::Int32]); + Ok(()) + } + #[test] fn test_variance_return_data_type() -> Result<()> { let data_type = DataType::Float64; diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index ef52a01e0598f..5baf64191f0f4 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -202,6 +202,7 @@ fn get_valid_types_with_aggregate_udf( _ => get_valid_types(signature, current_types)?, }; + println!("current types {:?}, valid_types: {:?}", current_types, valid_types); Ok(valid_types) } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index c3c37100ac1fe..66fdb95ff7296 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -46,6 +46,7 @@ use arrow::datatypes::{ DataType, Decimal128Type, Decimal256Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; +use arrow_schema::IntervalUnit; use datafusion_common::{downcast_value, internal_err, DataFusionError, Result}; use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; use std::fmt::Debug; @@ -62,31 +63,111 @@ use datafusion_common::ScalarValue; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDFImpl, Signature, Volatility, }; -use datafusion_expr::{Expr, GroupsAccumulator}; - +use datafusion_expr::{type_coercion, Expr, GroupsAccumulator}; + + + +pub static STRINGS: &[DataType] = &[DataType::Utf8, DataType::LargeUtf8]; + +pub static SIGNED_INTEGERS: &[DataType] = &[ + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, +]; + +pub static UNSIGNED_INTEGERS: &[DataType] = &[ + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, +]; + +pub static INTEGERS: &[DataType] = &[ + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, +]; + +pub static NUMERICS: &[DataType] = &[ + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, + DataType::Float32, + DataType::Float64, +]; + +pub static TIMESTAMPS: &[DataType] = &[ + DataType::Timestamp(TimeUnit::Second, None), + DataType::Timestamp(TimeUnit::Millisecond, None), + DataType::Timestamp(TimeUnit::Microsecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, None), +]; + +pub static DATES: &[DataType] = &[DataType::Date32, DataType::Date64]; + +pub static BINARYS: &[DataType] = &[DataType::Binary, DataType::LargeBinary]; + +pub static TIMES: &[DataType] = &[ + DataType::Time32(TimeUnit::Second), + DataType::Time32(TimeUnit::Millisecond), + DataType::Time64(TimeUnit::Microsecond), + DataType::Time64(TimeUnit::Nanosecond), +]; + +pub static TIMES_INTERVALS: &[DataType] = &[ + DataType::Interval(IntervalUnit::DayTime), + DataType::Interval(IntervalUnit::YearMonth), + DataType::Interval(IntervalUnit::MonthDayNano), +]; // Min/max aggregation can take Dictionary encode input but always produces unpacked // (aka non Dictionary) output. We need to adjust the output data type to reflect this. // The reason min/max aggregate produces unpacked output because there is only one // min/max value per group; there is no needs to keep them Dictionary encode -fn min_max_aggregate_data_type(input_type: DataType) -> DataType { +fn min_max_aggregate_data_type<'a>(input_type: &'a DataType) -> &'a DataType { if let DataType::Dictionary(_, value_type) = input_type { - *value_type + value_type } else { input_type } } + +fn min_max_signature() -> Signature { + let valid = STRINGS + .iter() + .chain(NUMERICS.iter()) + .chain(TIMESTAMPS.iter()) + .chain(DATES.iter()) + .chain(TIMES.iter()) + .chain(BINARYS.iter()) + .chain(TIMES_INTERVALS.iter()) + .cloned() + .collect::>(); + Signature::uniform(1, valid, Volatility::Immutable) +} + // MAX aggregate UDF #[derive(Debug)] pub struct Max { - signature: Signature, aliases: Vec, + signature: Signature } impl Max { pub fn new() -> Self { Self { - signature: Signature::numeric(1, Volatility::Immutable), aliases: vec!["max".to_owned()], + signature: min_max_signature(), } } } @@ -147,11 +228,20 @@ impl AggregateUDFImpl for Max { } fn return_type(&self, arg_types: &[DataType]) -> Result { - Ok(min_max_aggregate_data_type(arg_types[0].clone())) + type_coercion::aggregates::get_min_max_result_type(arg_types)? + .into_iter() + .next() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "Expected at one input type for MAX aggregate function" + )) + }) } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - Ok(Box::new(MaxAccumulator::try_new(acc_args.input_type)?)) + // let data_type = &min_max_aggregate_data_type(acc_args.data_type); + let data_type = acc_args.input_type; + Ok(Box::new(MaxAccumulator::try_new(data_type)?)) } fn aliases(&self) -> &[String] { @@ -159,8 +249,10 @@ impl AggregateUDFImpl for Max { } fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { + // let data_type = min_max_aggregate_data_type(args.data_type); + let data_type = args.input_type; matches!( - args.input_type, + data_type, DataType::Int8 | DataType::Int16 | DataType::Int32 @@ -187,6 +279,7 @@ impl AggregateUDFImpl for Max { ) -> Result> { use DataType::*; use TimeUnit::*; +// let data_type = min_max_aggregate_data_type(args.data_type); let data_type = args.input_type; match data_type { Int8 => instantiate_max_accumulator!(data_type, i8, Int8Type), @@ -248,12 +341,20 @@ impl AggregateUDFImpl for Max { &self, args: AccumulatorArgs, ) -> Result> { - Ok(Box::new(SlidingMaxAccumulator::try_new(args.input_type)?)) + let data_type = min_max_aggregate_data_type(args.data_type); + Ok(Box::new(SlidingMaxAccumulator::try_new(data_type)?)) } fn get_minmax_desc(&self) -> Option { Some(true) } + fn order_sensitivity(&self) -> datafusion_expr::utils::AggregateOrderSensitivity { + datafusion_expr::utils::AggregateOrderSensitivity::Insensitive + } + + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + type_coercion::aggregates::get_min_max_result_type(arg_types) + } } // Statically-typed version of min/max(array) -> ScalarValue for string types @@ -824,7 +925,7 @@ pub struct Min { impl Min { pub fn new() -> Self { Self { - signature: Signature::numeric(1, Volatility::Immutable), + signature: min_max_signature(), aliases: vec!["min".to_owned()], } } @@ -850,11 +951,23 @@ impl AggregateUDFImpl for Min { } fn return_type(&self, arg_types: &[DataType]) -> Result { - Ok(min_max_aggregate_data_type(arg_types[0].clone())) + let return_type = type_coercion::aggregates::get_min_max_result_type(arg_types)? + .into_iter() + .next() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "Expected at one input type for MAX aggregate function" + )) + }); + + println!("Return type for min {:?}", return_type); + return_type } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - Ok(Box::new(MinAccumulator::try_new(acc_args.input_type)?)) + Ok(Box::new(MinAccumulator::try_new( + &min_max_aggregate_data_type(acc_args.data_type), + )?)) } fn aliases(&self) -> &[String] { @@ -862,8 +975,9 @@ impl AggregateUDFImpl for Min { } fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { + let data_type = min_max_aggregate_data_type(args.data_type); matches!( - args.input_type, + data_type, DataType::Int8 | DataType::Int16 | DataType::Int32 @@ -890,7 +1004,7 @@ impl AggregateUDFImpl for Min { ) -> Result> { use DataType::*; use TimeUnit::*; - let data_type = args.input_type; + let data_type = min_max_aggregate_data_type(args.data_type); match data_type { Int8 => instantiate_min_accumulator!(data_type, i8, Int8Type), Int16 => instantiate_min_accumulator!(data_type, i16, Int16Type), @@ -951,12 +1065,21 @@ impl AggregateUDFImpl for Min { &self, args: AccumulatorArgs, ) -> Result> { - Ok(Box::new(SlidingMinAccumulator::try_new(args.input_type)?)) + let data_type = min_max_aggregate_data_type(args.data_type); + Ok(Box::new(SlidingMinAccumulator::try_new(data_type)?)) } fn get_minmax_desc(&self) -> Option { Some(false) } + + fn order_sensitivity(&self) -> datafusion_expr::utils::AggregateOrderSensitivity { + datafusion_expr::utils::AggregateOrderSensitivity::Insensitive + } + + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + type_coercion::aggregates::get_min_max_result_type(arg_types) + } } /// An accumulator to compute the minimum value #[derive(Debug)] @@ -965,7 +1088,7 @@ pub struct MinAccumulator { } impl MinAccumulator { - /// new max accumulator + /// new min accumulator pub fn try_new(datatype: &DataType) -> Result { Ok(Self { min: ScalarValue::try_from(datatype)?, @@ -1062,6 +1185,7 @@ impl Accumulator for SlidingMinAccumulator { std::mem::size_of_val(self) - std::mem::size_of_val(&self.min) + self.min.size() } } + // // Moving min and moving max // The implementation is taken from https://github.com/spebern/moving_min_max/blob/master/src/lib.rs. @@ -1454,4 +1578,23 @@ mod tests { moving_max_i32(100, 100)?; Ok(()) } + + #[test] + fn test_min_max_coerce_types() { + // the coerced types is same with input types + let funs: Vec> = + vec![Box::new(Min::new()), Box::new(Max::new())]; + let input_types = vec![ + vec![DataType::Int32], + vec![DataType::Decimal128(10, 2)], + vec![DataType::Decimal256(1, 1)], + vec![DataType::Utf8], + ]; + for fun in funs { + for input_type in &input_types { + let result = fun.coerce_types(input_type); + assert_eq!(*input_type, result.unwrap()); + } + } + } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt deleted file mode 100644 index fa228d499d1fc..0000000000000 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ /dev/null @@ -1,5520 +0,0 @@ -# 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. - -####### -# Setup test data table -####### -statement ok -CREATE EXTERNAL TABLE aggregate_test_100 ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 INT UNSIGNED NOT NULL, - c10 BIGINT UNSIGNED NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL -) -STORED AS CSV -LOCATION '../../testing/data/csv/aggregate_test_100.csv' -OPTIONS ('format.has_header' 'true'); - -statement ok -CREATE TABLE d_table (c1 decimal(10,3), c2 varchar) -as values -(110.000, 'A'), (110.001, 'A'), (110.002, 'A'), (110.003, 'A'), (110.004, 'A'), (110.005, 'A'), (110.006, 'A'), (110.007, 'A'), (110.008, 'A'), (110.009, 'A'), -(-100.000, 'B'),(-100.001, 'B'),(-100.002, 'B'),(-100.003, 'B'),(-100.004, 'B'),(-100.005, 'B'),(-100.006, 'B'),(-100.007, 'B'),(-100.008, 'B'),(-100.009, 'B') - -statement ok -CREATE TABLE median_table ( - col_i8 TINYINT, - col_i16 SMALLINT, - col_i32 INT, - col_i64 BIGINT, - col_u8 TINYINT UNSIGNED, - col_u16 SMALLINT UNSIGNED, - col_u32 INT UNSIGNED, - col_u64 BIGINT UNSIGNED, - col_f32 FLOAT, - col_f64 DOUBLE, - col_f64_nan DOUBLE -) as VALUES -( -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 1.1, 1.1, 1.1 ), -( -128, -32768, -2147483648, arrow_cast(-9223372036854775808,'Int64'), 0, 0, 0, arrow_cast(0,'UInt64'), 4.4, 4.4, arrow_cast('NAN','Float64') ), -( 100, 100, 100, arrow_cast(100,'Int64'), 100,100,100, arrow_cast(100,'UInt64'), 3.3, 3.3, arrow_cast('NAN','Float64') ), -( 127, 32767, 2147483647, arrow_cast(9223372036854775807,'Int64'), 255, 65535, 4294967295, 18446744073709551615, 2.2, 2.2, arrow_cast('NAN','Float64') ) - -statement ok -CREATE TABLE test (c1 BIGINT,c2 BIGINT) as values -(0,null), (1,1), (null,1), (3,2), (3,2) - -####### -# Error tests -####### - -# https://github.com/apache/datafusion/issues/3353 -statement error DataFusion error: Schema error: Schema contains duplicate unqualified field name "approx_distinct\(aggregate_test_100\.c9\)" -SELECT approx_distinct(c9) count_c9, approx_distinct(cast(c9 as varchar)) count_c9_str FROM aggregate_test_100 - -# csv_query_approx_percentile_cont_with_weight -statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Utf8, Int8, Float64\] to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont_with_weight(c1, c2, 0.95) FROM aggregate_test_100 - -statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Int16, Utf8, Float64\] to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont_with_weight(c3, c1, 0.95) FROM aggregate_test_100 - -statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Int16, Int8, Utf8\] to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont_with_weight(c3, c2, c1) FROM aggregate_test_100 - -# csv_query_approx_percentile_cont_with_histogram_bins -statement error DataFusion error: External error: This feature is not implemented: Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal \(got data type Int64\)\. -SELECT c1, approx_percentile_cont(c3, 0.95, -1000) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 - -statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Int16, Float64, Utf8\] to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont(c3, 0.95, c1) FROM aggregate_test_100 - -statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Int16, Float64, Float64\] to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont(c3, 0.95, 111.1) FROM aggregate_test_100 - -statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Float64, Float64, Float64\] to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont(c12, 0.95, 111.1) FROM aggregate_test_100 - -# array agg can use order by -query ? -SELECT array_agg(c13 ORDER BY c13) -FROM - (SELECT * - FROM aggregate_test_100 - ORDER BY c13 - LIMIT 5) as t1 ----- -[0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm, 0keZ5G8BffGwgF2RwQD59TFzMStxCB, 0og6hSkhbX8AC1ktFS4kounvTzy8Vo, 1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO, 2T3wSlHdEmASmO0xcXHnndkKEt6bz8] - -statement ok -CREATE EXTERNAL TABLE agg_order ( -c1 INT NOT NULL, -c2 INT NOT NULL, -c3 INT NOT NULL -) -STORED AS CSV -LOCATION '../core/tests/data/aggregate_agg_multi_order.csv' -OPTIONS ('format.has_header' 'true'); - -# test array_agg with order by multiple columns -query ? -select array_agg(c1 order by c2 desc, c3) from agg_order; ----- -[5, 6, 7, 8, 9, 1, 2, 3, 4, 10] - -query TT -explain select array_agg(c1 order by c2 desc, c3) from agg_order; ----- -logical_plan -01)Aggregate: groupBy=[[]], aggr=[[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] -02)--TableScan: agg_order projection=[c1, c2, c3] -physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] -02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] -04)------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST], preserve_partitioning=[true] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true - -# test array_agg_order with list data type -statement ok -CREATE TABLE array_agg_order_list_table AS VALUES - ('w', 2, [1,2,3], 10), - ('w', 1, [9,5,2], 20), - ('w', 1, [3,2,5], 30), - ('b', 2, [4,5,6], 20), - ('b', 1, [7,8,9], 30) -; - -query T? rowsort -select column1, array_agg(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; ----- -b [[7, 8, 9], [4, 5, 6]] -w [[3, 2, 5], [9, 5, 2], [1, 2, 3]] - -query T?? rowsort -select column1, first_value(column3 order by column2, column4 desc), last_value(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; ----- -b [7, 8, 9] [4, 5, 6] -w [3, 2, 5] [1, 2, 3] - -query T? rowsort -select column1, nth_value(column3, 2 order by column2, column4 desc) from array_agg_order_list_table group by column1; ----- -b [4, 5, 6] -w [9, 5, 2] - -statement ok -drop table array_agg_order_list_table; - -# test array_agg_distinct with list data type -statement ok -CREATE TABLE array_agg_distinct_list_table AS VALUES - ('w', [0,1]), - ('w', [0,1]), - ('w', [1,0]), - ('b', [1,0]), - ('b', [1,0]), - ('b', [1,0]), - ('b', [0,1]) -; - -# Apply array_sort to have deterministic result, higher dimension nested array also works but not for array sort, -# so they are covered in `datafusion/functions-aggregate/src/array_agg.rs` -query ?? -select array_sort(c1), array_sort(c2) from ( - select array_agg(distinct column1) as c1, array_agg(distinct column2) as c2 from array_agg_distinct_list_table -); ----- -[b, w] [[0, 1], [1, 0]] - -statement ok -drop table array_agg_distinct_list_table; - -statement error This feature is not implemented: Calling array_agg: LIMIT not supported in function arguments: 1 -SELECT array_agg(c13 LIMIT 1) FROM aggregate_test_100 - - -# Test distinct aggregate function with merge batch -query II -with A as ( - select 1 as id, 2 as foo - UNION ALL - select 1, null - UNION ALL - select 1, null - UNION ALL - select 1, 3 - UNION ALL - select 1, 2 - ---- The order is non-deterministic, verify with length -) select array_length(array_agg(distinct a.foo)), sum(distinct 1) from A a group by a.id; ----- -3 1 - -# It has only AggregateExec with FinalPartitioned mode, so `merge_batch` is used -# If the plan is changed, whether the `merge_batch` is used should be verified to ensure the test coverage -query TT -explain with A as ( - select 1 as id, 2 as foo - UNION ALL - select 1, null - UNION ALL - select 1, null - UNION ALL - select 1, 3 - UNION ALL - select 1, 2 -) select array_length(array_agg(distinct a.foo)), sum(distinct 1) from A a group by a.id; ----- -logical_plan -01)Projection: array_length(array_agg(DISTINCT a.foo)), sum(DISTINCT Int64(1)) -02)--Aggregate: groupBy=[[a.id]], aggr=[[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))]] -03)----SubqueryAlias: a -04)------SubqueryAlias: a -05)--------Union -06)----------Projection: Int64(1) AS id, Int64(2) AS foo -07)------------EmptyRelation -08)----------Projection: Int64(1) AS id, Int64(NULL) AS foo -09)------------EmptyRelation -10)----------Projection: Int64(1) AS id, Int64(NULL) AS foo -11)------------EmptyRelation -12)----------Projection: Int64(1) AS id, Int64(3) AS foo -13)------------EmptyRelation -14)----------Projection: Int64(1) AS id, Int64(2) AS foo -15)------------EmptyRelation -physical_plan -01)ProjectionExec: expr=[array_length(array_agg(DISTINCT a.foo)@1) as array_length(array_agg(DISTINCT a.foo)), sum(DISTINCT Int64(1))@2 as sum(DISTINCT Int64(1))] -02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([id@0], 4), input_partitions=5 -05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[array_agg(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted -06)----------UnionExec -07)------------ProjectionExec: expr=[1 as id, 2 as foo] -08)--------------PlaceholderRowExec -09)------------ProjectionExec: expr=[1 as id, NULL as foo] -10)--------------PlaceholderRowExec -11)------------ProjectionExec: expr=[1 as id, NULL as foo] -12)--------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as id, 3 as foo] -14)--------------PlaceholderRowExec -15)------------ProjectionExec: expr=[1 as id, 2 as foo] -16)--------------PlaceholderRowExec - - -# FIX: custom absolute values -# csv_query_avg_multi_batch - -# csv_query_avg -query R -SELECT avg(c12) FROM aggregate_test_100 ----- -0.508972509913 - -# csv_query_bit_and -query IIIII -SELECT bit_and(c5), bit_and(c6), bit_and(c7), bit_and(c8), bit_and(c9) FROM aggregate_test_100 ----- -0 0 0 0 0 - -# csv_query_bit_and_distinct -query IIIII -SELECT bit_and(distinct c5), bit_and(distinct c6), bit_and(distinct c7), bit_and(distinct c8), bit_and(distinct c9) FROM aggregate_test_100 ----- -0 0 0 0 0 - -# csv_query_bit_or -query IIIII -SELECT bit_or(c5), bit_or(c6), bit_or(c7), bit_or(c8), bit_or(c9) FROM aggregate_test_100 ----- --1 -1 255 65535 4294967295 - -# csv_query_bit_or_distinct -query IIIII -SELECT bit_or(distinct c5), bit_or(distinct c6), bit_or(distinct c7), bit_or(distinct c8), bit_or(distinct c9) FROM aggregate_test_100 ----- --1 -1 255 65535 4294967295 - -# csv_query_bit_xor -query IIIII -SELECT bit_xor(c5), bit_xor(c6), bit_xor(c7), bit_xor(c8), bit_xor(c9) FROM aggregate_test_100 ----- -1632751011 5960911605712039654 148 54789 169634700 - -# csv_query_bit_xor_distinct (should be different than above) -query IIIII -SELECT bit_xor(distinct c5), bit_xor(distinct c6), bit_xor(distinct c7), bit_xor(distinct c8), bit_xor(distinct c9) FROM aggregate_test_100 ----- -1632751011 5960911605712039654 196 54789 169634700 - -# csv_query_bit_xor_distinct_expr -query I -SELECT bit_xor(distinct c5 % 2) FROM aggregate_test_100 ----- --2 - -# csv_query_covariance_1 -query R -SELECT covar_pop(c2, c12) FROM aggregate_test_100 ----- --0.079169322354 - -# csv_query_covariance_2 -query R -SELECT covar(c2, c12) FROM aggregate_test_100 ----- --0.079969012479 - -# single_row_query_covar_1 -query R -select covar_samp(sq.column1, sq.column2) from (values (1.1, 2.2)) as sq ----- -NULL - -# single_row_query_covar_2 -query R -select covar_pop(sq.column1, sq.column2) from (values (1.1, 2.2)) as sq ----- -0 - -# all_nulls_query_covar -query RR -with data as ( - select null::int as f, null::int as b - union all - select null::int as f, null::int as b -) -select covar_samp(f, b), covar_pop(f, b) -from data ----- -NULL NULL - -# covar_query_with_nulls -query RR -with data as ( - select 1 as f, 4 as b - union all - select null as f, 99 as b - union all - select 2 as f, 5 as b - union all - select 98 as f, null as b - union all - select 3 as f, 6 as b - union all - select null as f, null as b -) -select covar_samp(f, b), covar_pop(f, b) -from data ----- -1 0.666666666667 - -# csv_query_correlation -query R -SELECT corr(c2, c12) FROM aggregate_test_100 ----- --0.190645441906 - -# single_row_query_correlation -query R -select corr(sq.column1, sq.column2) from (values (1.1, 2.2)) as sq ----- -0 - -# all_nulls_query_correlation -query R -with data as ( - select null::int as f, null::int as b - union all - select null::int as f, null::int as b -) -select corr(f, b) -from data ----- -NULL - -# correlation_query_with_nulls -query R -with data as ( - select 1 as f, 4 as b - union all - select null as f, 99 as b - union all - select 2 as f, 5 as b - union all - select 98 as f, null as b - union all - select 3 as f, 6 as b - union all - select null as f, null as b -) -select corr(f, b) -from data ----- -1 - -# csv_query_variance_1 -query R -SELECT var_pop(c2) FROM aggregate_test_100 ----- -1.8675 - -# csv_query_variance_2 -query R -SELECT var_pop(c6) FROM aggregate_test_100 ----- -26156334342021890000000000000000000000 - -# csv_query_variance_3 -query R -SELECT var_pop(c12) FROM aggregate_test_100 ----- -0.092342237216 - -# csv_query_variance_4 -query R -SELECT var(c2) FROM aggregate_test_100 ----- -1.886363636364 - -# csv_query_distinct_variance -query R -SELECT var(distinct c2) FROM aggregate_test_100 ----- -2.5 - -statement error DataFusion error: This feature is not implemented: VAR\(DISTINCT\) aggregations are not available -SELECT var(c2), var(distinct c2) FROM aggregate_test_100 - -# csv_query_distinct_variance_population -query R -SELECT var_pop(distinct c2) FROM aggregate_test_100 ----- -2 - -statement error DataFusion error: This feature is not implemented: VAR_POP\(DISTINCT\) aggregations are not available -SELECT var_pop(c2), var_pop(distinct c2) FROM aggregate_test_100 - -# csv_query_variance_5 -query R -SELECT var_samp(c2) FROM aggregate_test_100 ----- -1.886363636364 - -# csv_query_stddev_1 -query R -SELECT stddev_pop(c2) FROM aggregate_test_100 ----- -1.366565036872 - -# csv_query_stddev_2 -query R -SELECT stddev_pop(c6) FROM aggregate_test_100 ----- -5114326382039172000 - -# csv_query_stddev_3 -query R -SELECT stddev_pop(c12) FROM aggregate_test_100 ----- -0.303878655413 - -# csv_query_stddev_4 -query R -SELECT stddev(c12) FROM aggregate_test_100 ----- -0.305409539941 - -# csv_query_stddev_5 -query R -SELECT stddev_samp(c12) FROM aggregate_test_100 ----- -0.305409539941 - -# csv_query_stddev_6 -query R -select stddev(sq.column1) from (values (1.1), (2.0), (3.0)) as sq ----- -0.950438495292 - -# csv_query_approx_median_1 -query I -SELECT approx_median(c2) FROM aggregate_test_100 ----- -3 - -# csv_query_approx_median_2 -query I -SELECT approx_median(c6) FROM aggregate_test_100 ----- -1146409980542786560 - -# csv_query_approx_median_3 -query R -SELECT approx_median(c12) FROM aggregate_test_100 ----- -0.555006541052 - -# csv_query_approx_median_4 -# test with string, approx median only supports numeric -statement error -SELECT approx_median(c1) FROM aggregate_test_100 - -# csv_query_median_1 -query I -SELECT median(c2) FROM aggregate_test_100 ----- -3 - -# csv_query_median_2 -query I -SELECT median(c6) FROM aggregate_test_100 ----- -1125553990140691277 - -# csv_query_median_3 -query R -SELECT median(c12) FROM aggregate_test_100 ----- -0.551390054439 - -# median_i8 -query I -SELECT median(col_i8) FROM median_table ----- --14 - -# distinct_median_i8 -query I -SELECT median(distinct col_i8) FROM median_table ----- -100 - -query II -SELECT median(col_i8), median(distinct col_i8) FROM median_table ----- --14 100 - -# approx_distinct_median_i8 -query I -SELECT approx_median(distinct col_i8) FROM median_table ----- -100 - -statement error DataFusion error: This feature is not implemented: APPROX_MEDIAN\(DISTINCT\) aggregations are not available -SELECT approx_median(col_i8), approx_median(distinct col_i8) FROM median_table - -# median_i16 -query I -SELECT median(col_i16) FROM median_table ----- --16334 - -# median_i32 -query I -SELECT median(col_i32) FROM median_table ----- --1073741774 - -# median_i64 -query I -SELECT median(col_i64) FROM median_table ----- --4611686018427387854 - -# median_u8 -query I -SELECT median(col_u8) FROM median_table ----- -50 - -# median_u16 -query I -SELECT median(col_u16) FROM median_table ----- -50 - -# median_u32 -query I -SELECT median(col_u32) FROM median_table ----- -50 - -# median_u64 -query I -SELECT median(col_u64) FROM median_table ----- -50 - -# median_f32 -query R -SELECT median(col_f32) FROM median_table ----- -2.75 - -# median_f64 -query R -SELECT median(col_f64) FROM median_table ----- -2.75 - -# median_f64_nan -query R -SELECT median(col_f64_nan) FROM median_table ----- -NaN - -# approx_median_f64_nan -query R -SELECT approx_median(col_f64_nan) FROM median_table ----- -NaN - -# median decimal -statement ok -create table t(c decimal(10, 4)) as values (0.0001), (0.0002), (0.0003), (0.0004), (0.0005), (0.0006); - -query RT -select median(c), arrow_typeof(median(c)) from t; ----- -0.0003 Decimal128(10, 4) - -query RT -select approx_median(c), arrow_typeof(approx_median(c)) from t; ----- -0.00035 Float64 - -statement ok -drop table t; - -# median decimal with nulls -statement ok -create table t(c decimal(10, 4)) as values (0.0001), (null), (0.0003), (0.0004), (0.0005); - -query RT -select median(c), arrow_typeof(median(c)) from t; ----- -0.0003 Decimal128(10, 4) - -statement ok -drop table t; - -# median decimal with all nulls -statement ok -create table t(c decimal(10, 4)) as values (null), (null), (null); - -query RT -select median(c), arrow_typeof(median(c)) from t; ----- -NULL Decimal128(10, 4) - -statement ok -drop table t; - -# median odd -statement ok -create table t(c int) as values (1), (2), (3), (4), (5); - -query I -select median(c) from t; ----- -3 - -statement ok -drop table t; - -# median even -statement ok -create table t(c int) as values (1), (2), (3), (4), (5), (6); - -query I -select median(c) from t; ----- -3 - -statement ok -drop table t; - -# median with nulls -statement ok -create table t(c int) as values (1), (null), (3), (4), (5); - -query I -select median(c) from t; ----- -3 - -statement ok -drop table t; - -# median with all nulls -statement ok -create table t(c int) as values (null), (null), (null); - -query I -select median(c) from t; ----- -NULL - -statement ok -drop table t; - -# median u32 -statement ok -create table t(c int unsigned) as values (1), (2), (3), (4), (5); - -query I -select median(c) from t; ----- -3 - -statement ok -drop table t; - -# median f32 -statement ok -create table t(c float) as values (1.1), (2.2), (3.3), (4.4), (5.5); - -query R -select median(c) from t; ----- -3.3 - -statement ok -drop table t; - -# median distinct decimal -statement ok -create table t(c decimal(10, 4)) as values (0.0001), (0.0001), (0.0001), (0.0001), (0.0002), (0.0002), (0.0003), (0.0003); - -query R -select median(distinct c) from t; ----- -0.0002 - -statement ok -drop table t; - -# median distinct decimal with nulls -statement ok -create table t(c decimal(10, 4)) as values (0.0001), (0.0001), (0.0001), (null), (null), (0.0002), (0.0003), (0.0003); - -query R -select median(distinct c) from t; ----- -0.0002 - -statement ok -drop table t; - -# distinct median i32 odd -statement ok -create table t(c int) as values (2), (1), (1), (2), (1), (3); - -query I -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median i32 even -statement ok -create table t(c int) as values (1), (1), (3), (1), (1); - -query I -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median i32 with nulls -statement ok -create table t(c int) as values (1), (null), (1), (1), (3); - -query I -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median u32 odd -statement ok -create table t(c int unsigned) as values (1), (1), (2), (1), (3); - -query I -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median u32 even -statement ok -create table t(c int unsigned) as values (1), (1), (1), (1), (3), (3); - -query I -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median f32 odd -statement ok -create table t(c float) as values (3), (2), (1), (1), (1); - -query R -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median f32 even -statement ok -create table t(c float) as values (1), (1), (1), (1), (2); - -query R -select median(distinct c) from t; ----- -1.5 - -statement ok -drop table t; - -# distinct median f64 odd -statement ok -create table t(c double) as values (1), (1), (1), (2), (3); - -query R -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# distinct median f64 even -statement ok -create table t(c double) as values (1), (1), (1), (1), (2); - -query R -select median(distinct c) from t; ----- -1.5 - -statement ok -drop table t; - -# distinct median i32 -statement ok -create table t(c int) as values (1), (1), (1), (1), (2), (2), (3), (3); - -query I -select median(distinct c) from t; ----- -2 - -statement ok -drop table t; - -# optimize distinct median to group by -statement ok -create table t(c int) as values (1), (1), (1), (1), (2), (2), (3), (3); - -query TT -explain select median(distinct c) from t; ----- -logical_plan -01)Projection: median(alias1) AS median(DISTINCT t.c) -02)--Aggregate: groupBy=[[]], aggr=[[median(alias1)]] -03)----Aggregate: groupBy=[[t.c AS alias1]], aggr=[[]] -04)------TableScan: t projection=[c] -physical_plan -01)ProjectionExec: expr=[median(alias1)@0 as median(DISTINCT t.c)] -02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------MemoryExec: partitions=1, partition_sizes=[1] - -statement ok -drop table t; - -# median_multi -# test case for https://github.com/apache/datafusion/issues/3105 -# has an intermediate grouping -statement ok -create table cpu (host string, usage float) as select * from (values -('host0', 90.1), -('host1', 90.2), -('host1', 90.4) -); - -query TR rowsort -select host, median(usage) from cpu group by host; ----- -host0 90.1 -host1 90.3 - -statement ok -drop table cpu; - -# this test is to show create table as and select into works in the same way -statement ok -SELECT * INTO cpu -FROM (VALUES - ('host0', 90.1), - ('host1', 90.2), - ('host1', 90.4) - ) AS cpu (host, usage); - -query TR rowsort -select host, median(usage) from cpu group by host; ----- -host0 90.1 -host1 90.3 - -query R -select median(usage) from cpu; ----- -90.2 - -statement ok -drop table cpu; - -# median_multi_odd - -# data is not sorted and has an odd number of values per group -statement ok -create table cpu (host string, usage float) as select * from (values - ('host0', 90.2), - ('host1', 90.1), - ('host1', 90.5), - ('host0', 90.5), - ('host1', 90.0), - ('host1', 90.3), - ('host0', 87.9), - ('host1', 89.3) -); - -query TR rowsort -select host, median(usage) from cpu group by host; ----- -host0 90.2 -host1 90.1 - - -statement ok -drop table cpu; - -# median_multi_even -# data is not sorted and has an odd number of values per group -statement ok -create table cpu (host string, usage float) as select * from (values ('host0', 90.2), ('host1', 90.1), ('host1', 90.5), ('host0', 90.5), ('host1', 90.0), ('host1', 90.3), ('host1', 90.2), ('host1', 90.3)); - -query TR rowsort -select host, median(usage) from cpu group by host; ----- -host0 90.35 -host1 90.25 - -statement ok -drop table cpu - -# csv_query_external_table_count -query I -SELECT COUNT(c12) FROM aggregate_test_100 ----- -100 - -# csv_query_external_table_sum -query II -SELECT SUM(CAST(c7 AS BIGINT)), SUM(CAST(c8 AS BIGINT)) FROM aggregate_test_100 ----- -13060 3017641 - -# csv_query_count -query I -SELECT count(c12) FROM aggregate_test_100 ----- -100 - -# csv_query_count_distinct -query I -SELECT count(distinct c2) FROM aggregate_test_100 ----- -5 - -# csv_query_count_distinct_expr -query I -SELECT count(distinct c2 % 2) FROM aggregate_test_100 ----- -2 - -# csv_query_count_star -query I -SELECT COUNT(*) FROM aggregate_test_100 ----- -100 - -# csv_query_count_literal -query I -SELECT COUNT(2) FROM aggregate_test_100 ----- -100 - -# csv_query_approx_count -# FIX: https://github.com/apache/datafusion/issues/3353 -# query II -# SELECT approx_distinct(c9) AS count_c9, approx_distinct(cast(c9 as varchar)) count_c9_str FROM aggregate_test_100 -# ---- -# 100 99 - -# csv_query_approx_count_dupe_expr_aliased -query II -SELECT approx_distinct(c9) AS a, approx_distinct(c9) AS b FROM aggregate_test_100 ----- -100 100 - -## This test executes the APPROX_PERCENTILE_CONT aggregation against the test -## data, asserting the estimated quantiles are ±5% their actual values. -## -## Actual quantiles calculated with: -## -## ```r -## read_csv("./testing/data/csv/aggregate_test_100.csv") |> -## select_if(is.numeric) |> -## summarise_all(~ quantile(., c(0.1, 0.5, 0.9))) -## ``` -## -## Giving: -## -## ```text -## c2 c3 c4 c5 c6 c7 c8 c9 c10 c11 c12 -## -## 1 1 -95.3 -22925. -1882606710 -7.25e18 18.9 2671. 472608672. 1.83e18 0.109 0.0714 -## 2 3 15.5 4599 377164262 1.13e18 134. 30634 2365817608. 9.30e18 0.491 0.551 -## 3 5 102. 25334. 1991374996. 7.37e18 231 57518. 3776538487. 1.61e19 0.834 0.946 -## ``` -## -## Column `c12` is omitted due to a large relative error (~10%) due to the small -## float values. - -#csv_query_approx_percentile_cont (c2) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c2, 0.1) AS DOUBLE) / 1.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c2, 0.5) AS DOUBLE) / 3.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c2, 0.9) AS DOUBLE) / 5.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c3) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c3, 0.1) AS DOUBLE) / -95.3) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c3, 0.5) AS DOUBLE) / 15.5) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c3, 0.9) AS DOUBLE) / 102.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c4) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c4, 0.1) AS DOUBLE) / -22925.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c4, 0.5) AS DOUBLE) / 4599.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c4, 0.9) AS DOUBLE) / 25334.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c5) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c5, 0.1) AS DOUBLE) / -1882606710.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c5, 0.5) AS DOUBLE) / 377164262.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c5, 0.9) AS DOUBLE) / 1991374996.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c6) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c6, 0.1) AS DOUBLE) / -7250000000000000000) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c6, 0.5) AS DOUBLE) / 1130000000000000000) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c6, 0.9) AS DOUBLE) / 7370000000000000000) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c7) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c7, 0.1) AS DOUBLE) / 18.9) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c7, 0.5) AS DOUBLE) / 134.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c7, 0.9) AS DOUBLE) / 231.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c8) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c8, 0.1) AS DOUBLE) / 2671.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c8, 0.5) AS DOUBLE) / 30634.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c8, 0.9) AS DOUBLE) / 57518.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c9) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c9, 0.1) AS DOUBLE) / 472608672.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c9, 0.5) AS DOUBLE) / 2365817608.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c9, 0.9) AS DOUBLE) / 3776538487.0) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c10) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c10, 0.1) AS DOUBLE) / 1830000000000000000) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c10, 0.5) AS DOUBLE) / 9300000000000000000) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c10, 0.9) AS DOUBLE) / 16100000000000000000) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_approx_percentile_cont (c11) -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c11, 0.1) AS DOUBLE) / 0.109) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c11, 0.5) AS DOUBLE) / 0.491) < 0.05) AS q FROM aggregate_test_100 ----- -true - -query B -SELECT (ABS(1 - CAST(approx_percentile_cont(c11, 0.9) AS DOUBLE) / 0.834) < 0.05) AS q FROM aggregate_test_100 ----- -true - -# csv_query_cube_avg -query TIR -SELECT c1, c2, AVG(c3) FROM aggregate_test_100 GROUP BY CUBE (c1, c2) ORDER BY c1, c2 ----- -a 1 -17.6 -a 2 -15.333333333333 -a 3 -4.5 -a 4 -32 -a 5 -32 -a NULL -18.333333333333 -b 1 31.666666666667 -b 2 25.5 -b 3 -42 -b 4 -44.6 -b 5 -0.2 -b NULL -5.842105263158 -c 1 47.5 -c 2 -55.571428571429 -c 3 47.5 -c 4 -10.75 -c 5 12 -c NULL -1.333333333333 -d 1 -8.142857142857 -d 2 109.333333333333 -d 3 41.333333333333 -d 4 54 -d 5 -49.5 -d NULL 25.444444444444 -e 1 75.666666666667 -e 2 37.8 -e 3 48 -e 4 37.285714285714 -e 5 -11 -e NULL 40.333333333333 -NULL 1 16.681818181818 -NULL 2 8.363636363636 -NULL 3 20.789473684211 -NULL 4 1.260869565217 -NULL 5 -13.857142857143 -NULL NULL 7.81 - -# csv_query_rollup_avg -query TIIR -SELECT c1, c2, c3, AVG(c4) FROM aggregate_test_100 WHERE c1 IN ('a', 'b', NULL) GROUP BY ROLLUP (c1, c2, c3) ORDER BY c1, c2, c3 ----- -a 1 -85 -15154 -a 1 -56 8692 -a 1 -25 15295 -a 1 -5 12636 -a 1 83 -14704 -a 1 NULL 1353 -a 2 -48 -18025 -a 2 -43 13080 -a 2 45 15673 -a 2 NULL 3576 -a 3 -72 -11122 -a 3 -12 -9168 -a 3 13 22338.5 -a 3 14 28162 -a 3 17 -22796 -a 3 NULL 4958.833333333333 -a 4 -101 11640 -a 4 -54 -2376 -a 4 -38 20744 -a 4 65 -28462 -a 4 NULL 386.5 -a 5 -101 -12484 -a 5 -31 -12907 -a 5 36 -16974 -a 5 NULL -14121.666666666666 -a NULL NULL 306.047619047619 -b 1 12 7652 -b 1 29 -18218 -b 1 54 -18410 -b 1 NULL -9658.666666666666 -b 2 -60 -21739 -b 2 31 23127 -b 2 63 21456 -b 2 68 15874 -b 2 NULL 9679.5 -b 3 -101 -13217 -b 3 17 14457 -b 3 NULL 620 -b 4 -117 19316 -b 4 -111 -1967 -b 4 -59 25286 -b 4 17 -28070 -b 4 47 20690 -b 4 NULL 7051 -b 5 -82 22080 -b 5 -44 15788 -b 5 -5 24896 -b 5 62 16337 -b 5 68 21576 -b 5 NULL 20135.4 -b NULL NULL 7732.315789473684 -NULL NULL NULL 3833.525 - -# csv_query_groupingsets_avg -query TIIR -SELECT c1, c2, c3, AVG(c4) -FROM aggregate_test_100 -WHERE c1 IN ('a', 'b', NULL) -GROUP BY GROUPING SETS ((c1), (c1,c2), (c1,c2,c3)) -ORDER BY c1, c2, c3 ----- -a 1 -85 -15154 -a 1 -56 8692 -a 1 -25 15295 -a 1 -5 12636 -a 1 83 -14704 -a 1 NULL 1353 -a 2 -48 -18025 -a 2 -43 13080 -a 2 45 15673 -a 2 NULL 3576 -a 3 -72 -11122 -a 3 -12 -9168 -a 3 13 22338.5 -a 3 14 28162 -a 3 17 -22796 -a 3 NULL 4958.833333333333 -a 4 -101 11640 -a 4 -54 -2376 -a 4 -38 20744 -a 4 65 -28462 -a 4 NULL 386.5 -a 5 -101 -12484 -a 5 -31 -12907 -a 5 36 -16974 -a 5 NULL -14121.666666666666 -a NULL NULL 306.047619047619 -b 1 12 7652 -b 1 29 -18218 -b 1 54 -18410 -b 1 NULL -9658.666666666666 -b 2 -60 -21739 -b 2 31 23127 -b 2 63 21456 -b 2 68 15874 -b 2 NULL 9679.5 -b 3 -101 -13217 -b 3 17 14457 -b 3 NULL 620 -b 4 -117 19316 -b 4 -111 -1967 -b 4 -59 25286 -b 4 17 -28070 -b 4 47 20690 -b 4 NULL 7051 -b 5 -82 22080 -b 5 -44 15788 -b 5 -5 24896 -b 5 62 16337 -b 5 68 21576 -b 5 NULL 20135.4 -b NULL NULL 7732.315789473684 - -# csv_query_singlecol_with_rollup_avg -query TIIR -SELECT c1, c2, c3, AVG(c4) -FROM aggregate_test_100 -WHERE c1 IN ('a', 'b', NULL) -GROUP BY c1, ROLLUP (c2, c3) -ORDER BY c1, c2, c3 ----- -a 1 -85 -15154 -a 1 -56 8692 -a 1 -25 15295 -a 1 -5 12636 -a 1 83 -14704 -a 1 NULL 1353 -a 2 -48 -18025 -a 2 -43 13080 -a 2 45 15673 -a 2 NULL 3576 -a 3 -72 -11122 -a 3 -12 -9168 -a 3 13 22338.5 -a 3 14 28162 -a 3 17 -22796 -a 3 NULL 4958.833333333333 -a 4 -101 11640 -a 4 -54 -2376 -a 4 -38 20744 -a 4 65 -28462 -a 4 NULL 386.5 -a 5 -101 -12484 -a 5 -31 -12907 -a 5 36 -16974 -a 5 NULL -14121.666666666666 -a NULL NULL 306.047619047619 -b 1 12 7652 -b 1 29 -18218 -b 1 54 -18410 -b 1 NULL -9658.666666666666 -b 2 -60 -21739 -b 2 31 23127 -b 2 63 21456 -b 2 68 15874 -b 2 NULL 9679.5 -b 3 -101 -13217 -b 3 17 14457 -b 3 NULL 620 -b 4 -117 19316 -b 4 -111 -1967 -b 4 -59 25286 -b 4 17 -28070 -b 4 47 20690 -b 4 NULL 7051 -b 5 -82 22080 -b 5 -44 15788 -b 5 -5 24896 -b 5 62 16337 -b 5 68 21576 -b 5 NULL 20135.4 -b NULL NULL 7732.315789473684 - -# csv_query_approx_percentile_cont_with_weight -query TI -SELECT c1, approx_percentile_cont(c3, 0.95) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ----- -a 73 -b 68 -c 122 -d 124 -e 115 - -# csv_query_approx_percentile_cont_with_weight (2) -query TI -SELECT c1, approx_percentile_cont_with_weight(c3, 1, 0.95) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ----- -a 73 -b 68 -c 122 -d 124 -e 115 - -# csv_query_approx_percentile_cont_with_histogram_bins -query TI -SELECT c1, approx_percentile_cont(c3, 0.95, 200) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ----- -a 73 -b 68 -c 122 -d 124 -e 115 - -query TI -SELECT c1, approx_percentile_cont_with_weight(c3, c2, 0.95) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ----- -a 74 -b 68 -c 123 -d 124 -e 115 - -# csv_query_sum_crossjoin -query TTI -SELECT a.c1, b.c1, SUM(a.c2) FROM aggregate_test_100 as a CROSS JOIN aggregate_test_100 as b GROUP BY a.c1, b.c1 ORDER BY a.c1, b.c1 ----- -a a 1260 -a b 1140 -a c 1260 -a d 1080 -a e 1260 -b a 1302 -b b 1178 -b c 1302 -b d 1116 -b e 1302 -c a 1176 -c b 1064 -c c 1176 -c d 1008 -c e 1176 -d a 924 -d b 836 -d c 924 -d d 792 -d e 924 -e a 1323 -e b 1197 -e c 1323 -e d 1134 -e e 1323 - -# csv_query_cube_sum_crossjoin -query TTI -SELECT a.c1, b.c1, SUM(a.c2) FROM aggregate_test_100 as a CROSS JOIN aggregate_test_100 as b GROUP BY CUBE (a.c1, b.c1) ORDER BY a.c1, b.c1 ----- -a a 1260 -a b 1140 -a c 1260 -a d 1080 -a e 1260 -a NULL 6000 -b a 1302 -b b 1178 -b c 1302 -b d 1116 -b e 1302 -b NULL 6200 -c a 1176 -c b 1064 -c c 1176 -c d 1008 -c e 1176 -c NULL 5600 -d a 924 -d b 836 -d c 924 -d d 792 -d e 924 -d NULL 4400 -e a 1323 -e b 1197 -e c 1323 -e d 1134 -e e 1323 -e NULL 6300 -NULL a 5985 -NULL b 5415 -NULL c 5985 -NULL d 5130 -NULL e 5985 -NULL NULL 28500 - -# csv_query_cube_distinct_count -query TII -SELECT c1, c2, COUNT(DISTINCT c3) FROM aggregate_test_100 GROUP BY CUBE (c1,c2) ORDER BY c1,c2 ----- -a 1 5 -a 2 3 -a 3 5 -a 4 4 -a 5 3 -a NULL 19 -b 1 3 -b 2 4 -b 3 2 -b 4 5 -b 5 5 -b NULL 17 -c 1 4 -c 2 7 -c 3 4 -c 4 4 -c 5 2 -c NULL 21 -d 1 7 -d 2 3 -d 3 3 -d 4 3 -d 5 2 -d NULL 18 -e 1 3 -e 2 4 -e 3 4 -e 4 7 -e 5 2 -e NULL 18 -NULL 1 22 -NULL 2 20 -NULL 3 17 -NULL 4 23 -NULL 5 14 -NULL NULL 80 - -# csv_query_rollup_distinct_count -query TII -SELECT c1, c2, COUNT(DISTINCT c3) FROM aggregate_test_100 GROUP BY ROLLUP (c1,c2) ORDER BY c1,c2 ----- -a 1 5 -a 2 3 -a 3 5 -a 4 4 -a 5 3 -a NULL 19 -b 1 3 -b 2 4 -b 3 2 -b 4 5 -b 5 5 -b NULL 17 -c 1 4 -c 2 7 -c 3 4 -c 4 4 -c 5 2 -c NULL 21 -d 1 7 -d 2 3 -d 3 3 -d 4 3 -d 5 2 -d NULL 18 -e 1 3 -e 2 4 -e 3 4 -e 4 7 -e 5 2 -e NULL 18 -NULL NULL 80 - -# csv_query_rollup_sum_crossjoin -query TTI -SELECT a.c1, b.c1, SUM(a.c2) FROM aggregate_test_100 as a CROSS JOIN aggregate_test_100 as b GROUP BY ROLLUP (a.c1, b.c1) ORDER BY a.c1, b.c1 ----- -a a 1260 -a b 1140 -a c 1260 -a d 1080 -a e 1260 -a NULL 6000 -b a 1302 -b b 1178 -b c 1302 -b d 1116 -b e 1302 -b NULL 6200 -c a 1176 -c b 1064 -c c 1176 -c d 1008 -c e 1176 -c NULL 5600 -d a 924 -d b 836 -d c 924 -d d 792 -d e 924 -d NULL 4400 -e a 1323 -e b 1197 -e c 1323 -e d 1134 -e e 1323 -e NULL 6300 -NULL NULL 28500 - -# query_count_without_from -query I -SELECT count(1 + 1) ----- -1 - -# csv_query_array_agg -query ? -SELECT array_agg(c13) FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 2) test ----- -[0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm, 0keZ5G8BffGwgF2RwQD59TFzMStxCB] - -# csv_query_array_agg_empty -query ? -SELECT array_agg(c13) FROM (SELECT * FROM aggregate_test_100 LIMIT 0) test ----- -NULL - -# csv_query_array_agg_one -query ? -SELECT array_agg(c13) FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) test ----- -[0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm] - -# csv_query_array_agg_with_overflow -query IIRIII -select c2, sum(c3) sum_c3, avg(c3) avg_c3, max(c3) max_c3, min(c3) min_c3, count(c3) count_c3 from aggregate_test_100 group by c2 order by c2 ----- -1 367 16.681818181818 125 -99 22 -2 184 8.363636363636 122 -117 22 -3 395 20.789473684211 123 -101 19 -4 29 1.260869565217 123 -117 23 -5 -194 -13.857142857143 118 -101 14 - -# csv_query_array_cube_agg_with_overflow -query TIIRIII -select c1, c2, sum(c3) sum_c3, avg(c3) avg_c3, max(c3) max_c3, min(c3) min_c3, count(c3) count_c3 from aggregate_test_100 group by CUBE (c1,c2) order by c1, c2 ----- -a 1 -88 -17.6 83 -85 5 -a 2 -46 -15.333333333333 45 -48 3 -a 3 -27 -4.5 17 -72 6 -a 4 -128 -32 65 -101 4 -a 5 -96 -32 36 -101 3 -a NULL -385 -18.333333333333 83 -101 21 -b 1 95 31.666666666667 54 12 3 -b 2 102 25.5 68 -60 4 -b 3 -84 -42 17 -101 2 -b 4 -223 -44.6 47 -117 5 -b 5 -1 -0.2 68 -82 5 -b NULL -111 -5.842105263158 68 -117 19 -c 1 190 47.5 103 -24 4 -c 2 -389 -55.571428571429 29 -117 7 -c 3 190 47.5 97 -2 4 -c 4 -43 -10.75 123 -90 4 -c 5 24 12 118 -94 2 -c NULL -28 -1.333333333333 123 -117 21 -d 1 -57 -8.142857142857 125 -99 7 -d 2 328 109.333333333333 122 93 3 -d 3 124 41.333333333333 123 -76 3 -d 4 162 54 102 5 3 -d 5 -99 -49.5 -40 -59 2 -d NULL 458 25.444444444444 125 -99 18 -e 1 227 75.666666666667 120 36 3 -e 2 189 37.8 97 -61 5 -e 3 192 48 112 -95 4 -e 4 261 37.285714285714 97 -56 7 -e 5 -22 -11 64 -86 2 -e NULL 847 40.333333333333 120 -95 21 -NULL 1 367 16.681818181818 125 -99 22 -NULL 2 184 8.363636363636 122 -117 22 -NULL 3 395 20.789473684211 123 -101 19 -NULL 4 29 1.260869565217 123 -117 23 -NULL 5 -194 -13.857142857143 118 -101 14 -NULL NULL 781 7.81 125 -117 100 - -# select with count to forces array_agg_distinct function, since single distinct expression is converted to group by by optimizer -# csv_query_array_agg_distinct -query ?I -SELECT array_sort(array_agg(distinct c2)), count(1) FROM aggregate_test_100 ----- -[1, 2, 3, 4, 5] 100 - -# aggregate_time_min_and_max -query TT -select min(t), max(t) from (select '00:00:00' as t union select '00:00:01' union select '00:00:02') ----- -00:00:00 00:00:02 - -# aggregate Interval(MonthDayNano) min/max -query T?? -select - arrow_typeof(min(column1)), min(column1), max(column1) -from values - (interval '1 month'), - (interval '2 months'), - (interval '2 month 15 days'), - (interval '-2 month') ----- -Interval(MonthDayNano) 0 years -2 mons 0 days 0 hours 0 mins 0.000000000 secs 0 years 2 mons 15 days 0 hours 0 mins 0.000000000 secs - -# aggregate Interval(DayTime) min/max -query T?? -select - arrow_typeof(min(column1)), min(column1), max(column1) -from values - (arrow_cast('60 minutes', 'Interval(DayTime)')), - (arrow_cast('-3 minutes', 'Interval(DayTime)')), - (arrow_cast('30 minutes', 'Interval(DayTime)')); ----- -Interval(DayTime) 0 years 0 mons 0 days 0 hours -3 mins 0.000 secs 0 years 0 mons 0 days 1 hours 0 mins 0.000 secs - -# aggregate Interval(YearMonth) min/max -query T?? -select - arrow_typeof(min(column1)), min(column1), max(column1) -from values - (arrow_cast('-1 year', 'Interval(YearMonth)')), - (arrow_cast('13 months', 'Interval(YearMonth)')), - (arrow_cast('1 year', 'Interval(YearMonth)')); ----- -Interval(YearMonth) -1 years 0 mons 0 days 0 hours 0 mins 0.00 secs 1 years 1 mons 0 days 0 hours 0 mins 0.00 secs - -# aggregate -query II -SELECT SUM(c1), SUM(c2) FROM test ----- -7 6 - -# aggregate_empty - -query II -SELECT SUM(c1), SUM(c2) FROM test where c1 > 100000 ----- -NULL NULL - -# aggregate_avg -query RR -SELECT AVG(c1), AVG(c2) FROM test ----- -1.75 1.5 - -# aggregate_max -query II -SELECT MAX(c1), MAX(c2) FROM test ----- -3 2 - -# aggregate_min -query II -SELECT MIN(c1), MIN(c2) FROM test ----- -0 1 - -# aggregate_grouped -query II -SELECT c1, SUM(c2) FROM test GROUP BY c1 order by c1 ----- -0 NULL -1 1 -3 4 -NULL 1 - -# aggregate_grouped_avg -query IR -SELECT c1, AVG(c2) FROM test GROUP BY c1 order by c1 ----- -0 NULL -1 1 -3 2 -NULL 1 - -# aggregate_grouped_empty -query IR -SELECT c1, AVG(c2) FROM test WHERE c1 = 123 GROUP BY c1 ----- - -# aggregate_grouped_max -query II -SELECT c1, MAX(c2) FROM test GROUP BY c1 order by c1 ----- -0 NULL -1 1 -3 2 -NULL 1 - -# aggregate_grouped_min -query II -SELECT c1, MIN(c2) FROM test GROUP BY c1 order by c1 ----- -0 NULL -1 1 -3 2 -NULL 1 - -# aggregate_min_max_w_custom_window_frames -query RR -SELECT -MIN(c12) OVER (ORDER BY C12 RANGE BETWEEN 0.3 PRECEDING AND 0.2 FOLLOWING) as min1, -MAX(c12) OVER (ORDER BY C11 RANGE BETWEEN 0.1 PRECEDING AND 0.2 FOLLOWING) as max1 -FROM aggregate_test_100 -ORDER BY C9 -LIMIT 5 ----- -0.014793053078 0.996540038759 -0.014793053078 0.980019341044 -0.014793053078 0.970671228336 -0.266717779508 0.996540038759 -0.360076636233 0.970671228336 - -# aggregate_min_max_with_custom_window_frames_unbounded_start -query RR -SELECT -MIN(c12) OVER (ORDER BY C12 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.2 FOLLOWING) as min1, -MAX(c12) OVER (ORDER BY C11 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.2 FOLLOWING) as max1 -FROM aggregate_test_100 -ORDER BY C9 -LIMIT 5 ----- -0.014793053078 0.996540038759 -0.014793053078 0.980019341044 -0.014793053078 0.980019341044 -0.014793053078 0.996540038759 -0.014793053078 0.980019341044 - -# aggregate_avg_add -query RRRR -SELECT AVG(c1), AVG(c1) + 1, AVG(c1) + 2, 1 + AVG(c1) FROM test ----- -1.75 2.75 3.75 2.75 - -# case_sensitive_identifiers_aggregates -query I -SELECT max(c1) FROM test; ----- -3 - - - -# count_basic -statement ok -create table t (c int) as values (1), (2), (null), (3), (null), (4), (5); - -query IT -select count(c), arrow_typeof(count(c)) from t; ----- -5 Int64 - -statement ok -drop table t; - -# test count with all nulls -statement ok -create table t (c int) as values (null), (null), (null), (null), (null); - -query IT -select count(c), arrow_typeof(count(c)) from t; ----- -0 Int64 - -statement ok -drop table t; - -# test with empty -statement ok -create table t (c int); - -query IT -select count(c), arrow_typeof(count(c)) from t; ----- -0 Int64 - -statement ok -drop table t; - -# test count with string -statement ok -create table t (c string) as values ('a'), ('b'), (null), ('c'), (null), ('d'), ('e'); - -query IT -select count(c), arrow_typeof(count(c)) from t; ----- -5 Int64 - -statement ok -drop table t; - -# test count with largeutf8 -statement ok -create table t (c string) as values - (arrow_cast('a', 'LargeUtf8')), - (arrow_cast('b', 'LargeUtf8')), - (arrow_cast(null, 'LargeUtf8')), - (arrow_cast('c', 'LargeUtf8')) -; - -query T -select arrow_typeof(c) from t; ----- -Utf8 -Utf8 -Utf8 -Utf8 - -query IT -select count(c), arrow_typeof(count(c)) from t; ----- -3 Int64 - -statement ok -drop table t; - -# test count with multiple columns -statement ok -create table t (c1 int, c2 int) as values (1, 1), (2, null), (null, 2), (null, null), (3, 3), (null, 4); - -query IT -select count(c1, c2), arrow_typeof(count(c1, c2)) from t; ----- -2 Int64 - -statement ok -drop table t; - - -query II -SELECT COUNT(c1), COUNT(c2) FROM test ----- -4 4 - -# TODO: count_partitioned - -# TODO: count_aggregated - -# TODO: count_aggregated_cube - -# count_multi_expr -query I -SELECT count(c1, c2) FROM test ----- -3 - -# count(distinct) with multiple arguments -query error DataFusion error: This feature is not implemented: COUNT DISTINCT with multiple arguments -SELECT count(distinct c1, c2) FROM test - -# count_null -query III -SELECT count(null), count(null, null), count(distinct null) FROM test ----- -0 0 0 - -# count_multi_expr_group_by -query I -SELECT count(c1, c2) FROM test group by c1 order by c1 ----- -0 -1 -2 -0 - -# count_null_group_by -query III -SELECT count(null), count(null, null), count(distinct null) FROM test group by c1 order by c1 ----- -0 0 0 -0 0 0 -0 0 0 -0 0 0 - -# aggreggte_with_alias -query II -select c1, sum(c2) as `Total Salary` from test group by c1 order by c1 ----- -0 NULL -1 1 -3 4 -NULL 1 - -# simple_avg - -query R -select avg(c1) from test ----- -1.75 - -# avg_decimal -statement ok -create table t (c1 decimal(10, 0)) as values (1), (2), (3), (4), (5), (6); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3.5 Decimal128(14, 4) - -statement ok -drop table t; - -# avg_decimal_with_nulls -statement ok -create table t (c1 decimal(10, 0)) as values (1), (NULL), (3), (4), (5); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3.25 Decimal128(14, 4) - -statement ok -drop table t; - -# avg_decimal_all_nulls -statement ok -create table t (c1 decimal(10, 0)) as values (NULL), (NULL), (NULL), (NULL), (NULL), (NULL); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -NULL Decimal128(14, 4) - -statement ok -drop table t; - -# avg_i32 -statement ok -create table t (c1 int) as values (1), (2), (3), (4), (5); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3 Float64 - -statement ok -drop table t; - -# avg_i32_with_nulls -statement ok -create table t (c1 int) as values (1), (NULL), (3), (4), (5); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3.25 Float64 - -statement ok -drop table t; - -# avg_i32_all_nulls -statement ok -create table t (c1 int) as values (NULL), (NULL); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# avg_u32 -statement ok -create table t (c1 int unsigned) as values (1), (2), (3), (4), (5); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3 Float64 - -statement ok -drop table t; - -# avg_f32 -statement ok -create table t (c1 float) as values (1), (2), (3), (4), (5); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3 Float64 - -statement ok -drop table t; - -# avg_f64 -statement ok -create table t (c1 double) as values (1), (2), (3), (4), (5); - -query RT -select avg(c1), arrow_typeof(avg(c1)) from t; ----- -3 Float64 - -statement ok -drop table t; - -# covariance_f64_1 -statement ok -create table t (c1 double, c2 double) as values (1, 4), (2, 5), (3, 6); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.666666666667 Float64 - -statement ok -drop table t; - -# covariance_f64_2 -statement ok -create table t (c1 double, c2 double) as values (1, 4), (2, 5), (3, 6); - -query RT -select covar_samp(c1, c2), arrow_typeof(covar_samp(c1, c2)) from t; ----- -1 Float64 - -statement ok -drop table t; - -# covariance_f64_4 -statement ok -create table t (c1 double, c2 double) as values (1.1, 4.1), (2.0, 5.0), (3.0, 6.0); - -query RT -select covar_samp(c1, c2), arrow_typeof(covar_samp(c1, c2)) from t; ----- -0.903333333333 Float64 - -statement ok -drop table t; - -# covariance_f64_5 -statement ok -create table t (c1 double, c2 double) as values (1.1, 4.1), (2.0, 5.0), (3.0, 6.0); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.602222222222 Float64 - -statement ok -drop table t; - -# covariance_f64_6 -statement ok -create table t (c1 double, c2 double) as values (1.0, 4.0), (2.0, 5.0), (3.0, 6.0), (1.1, 4.4), (2.2, 5.5), (3.3, 6.6); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.761666666667 Float64 - -statement ok -drop table t; - -# covariance_i32 -statement ok -create table t (c1 int, c2 int) as values (1, 4), (2, 5), (3, 6); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.666666666667 Float64 - -statement ok -drop table t; - -# covariance_u32 -statement ok -create table t (c1 int unsigned, c2 int unsigned) as values (1, 4), (2, 5), (3, 6); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.666666666667 Float64 - -statement ok -drop table t; - -# covariance_f32 -statement ok -create table t (c1 float, c2 float) as values (1, 4), (2, 5), (3, 6); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.666666666667 Float64 - -statement ok -drop table t; - -# covariance_i32_with_nulls_1 -statement ok -create table t (c1 int, c2 int) as values (1, 4), (null, null), (3, 6); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -1 Float64 - -statement ok -drop table t; - -# covariance_i32_with_nulls_2 -statement ok -create table t (c1 int, c2 int) as values (1, 4), (null, 9), (2, 5), (null, 8), (3, 6), (null, null); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0.666666666667 Float64 - -statement ok -drop table t; - -# covariance_i32_with_nulls_3 -statement ok -create table t (c1 int, c2 int) as values (1, 4), (null, 9), (2, 5), (null, 8), (3, 6), (null, null); - -query RT -select covar_samp(c1, c2), arrow_typeof(covar_samp(c1, c2)) from t; ----- -1 Float64 - -statement ok -drop table t; - -# covariance_i32_all_nulls -statement ok -create table t (c1 int, c2 int) as values (null, null), (null, null); - -query RT -select covar_samp(c1, c2), arrow_typeof(covar_samp(c1, c2)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# covariance_pop_i32_all_nulls -statement ok -create table t (c1 int, c2 int) as values (null, null), (null, null); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# covariance_1_input -statement ok -create table t (c1 double, c2 double) as values (1, 2); - -query RT -select covar_samp(c1, c2), arrow_typeof(covar_samp(c1, c2)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# covariance_pop_1_input -statement ok -create table t (c1 double, c2 double) as values (1, 2); - -query RT -select covar_pop(c1, c2), arrow_typeof(covar_pop(c1, c2)) from t; ----- -0 Float64 - -statement ok -drop table t; - -# variance_f64_1 -statement ok -create table t (c double) as values (1), (2), (3), (4), (5); - -query RT -select var(c), arrow_typeof(var(c)) from t; ----- -2.5 Float64 - -statement ok -drop table t; - -# aggregate stddev f64_1 -statement ok -create table t (c1 double) as values (1), (2); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -0.5 Float64 - -statement ok -drop table t; - -# aggregate stddev f64_2 -statement ok -create table t (c1 double) as values (1.1), (2), (3); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -0.776029781788 Float64 - -statement ok -drop table t; - -# aggregate stddev f64_3 -statement ok -create table t (c1 double) as values (1), (2), (3), (4), (5); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -1.414213562373 Float64 - -statement ok -drop table t; - -# aggregate stddev f64_4 -statement ok -create table t (c1 double) as values (1.1), (2), (3); - -query RT -select stddev(c1), arrow_typeof(stddev(c1)) from t; ----- -0.950438495292 Float64 - -statement ok -drop table t; - -# aggregate stddev i32 -statement ok -create table t (c1 int) as values (1), (2), (3), (4), (5); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -1.414213562373 Float64 - -statement ok -drop table t; - -# aggregate stddev u32 -statement ok -create table t (c1 int unsigned) as values (1), (2), (3), (4), (5); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -1.414213562373 Float64 - -statement ok -drop table t; - -# aggregate stddev f32 -statement ok -create table t (c1 float) as values (1), (2), (3), (4), (5); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -1.414213562373 Float64 - -statement ok -drop table t; - -# aggregate stddev single_input -statement ok -create table t (c1 double) as values (1); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -0 Float64 - -statement ok -drop table t; - -# aggregate stddev with_nulls -statement ok -create table t (c1 int) as values (1), (null), (3), (4), (5); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -1.479019945775 Float64 - -statement ok -drop table t; - -# aggregate stddev all_nulls -statement ok -create table t (c1 int) as values (null), (null); - -query RT -select stddev_pop(c1), arrow_typeof(stddev_pop(c1)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# aggregate variance f64_1 -statement ok -create table t (c1 double) as values (1), (2); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -0.25 Float64 - -statement ok -drop table t; - -# aggregate variance f64_2 -statement ok -create table t (c1 double) as values (1), (2), (3), (4), (5); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -2 Float64 - -statement ok -drop table t; - -# aggregate variance f64_3 -statement ok -create table t (c1 double) as values (1), (2), (3), (4), (5); - -query RT -select var(c1), arrow_typeof(var(c1)) from t; ----- -2.5 Float64 - -statement ok -drop table t; - -# variance_f64_2 -statement ok -create table t (c double) as values (1.1), (2), (3); - -query RT -select var(c), arrow_typeof(var(c)) from t; ----- -0.903333333333 Float64 - -statement ok -drop table t; - -# aggregate variance f64_4 -statement ok -create table t (c1 double) as values (1.1), (2), (3); - -query RT -select var(c1), arrow_typeof(var(c1)) from t; ----- -0.903333333333 Float64 - -statement ok -drop table t; - -# variance_1_input -statement ok -create table t (a double not null) as values (1); - -query RT -select var(a), arrow_typeof(var(a)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# variance_i32_all_nulls -statement ok -create table t (a int) as values (null), (null); - -query RT -select var(a), arrow_typeof(var(a)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# aggregate variance i32 -statement ok -create table t (c1 int) as values (1), (2), (3), (4), (5); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -2 Float64 - -statement ok -drop table t; - -# aggregate variance u32 -statement ok -create table t (c1 int unsigned) as values (1), (2), (3), (4), (5); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -2 Float64 - -statement ok -drop table t; - -# aggregate variance f32 -statement ok -create table t (c1 float) as values (1), (2), (3), (4), (5); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -2 Float64 - -statement ok -drop table t; - -# aggregate single input -statement ok -create table t (c1 double) as values (1); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -0 Float64 - -statement ok -drop table t; - -# aggregate i32 with nulls -statement ok -create table t (c1 int) as values (1), (null), (3), (4), (5); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -2.1875 Float64 - -statement ok -drop table t; - -# aggregate i32 all nulls -statement ok -create table t (c1 int) as values (null), (null); - -query RT -select var_pop(c1), arrow_typeof(var_pop(c1)) from t; ----- -NULL Float64 - -statement ok -drop table t; - -# simple_mean -query R -select mean(c1) from test ----- -1.75 - -# aggregate sum distinct, coerced result from i32 to i64 -statement ok -create table t (c int) as values (1), (2), (1), (3), (null), (null), (-3), (-3); - -query IT -select sum(distinct c), arrow_typeof(sum(distinct c)) from t; ----- -3 Int64 - -statement ok -drop table t; - -# aggregate sum distinct, coerced result from u32 to u64 -statement ok -create table t (c int unsigned) as values (1), (2), (1), (3), (null), (null), (3); - -query IT -select sum(distinct c), arrow_typeof(sum(distinct c)) from t; ----- -6 UInt64 - -statement ok -drop table t; - -# aggregate sum distinct, coerced result from f32 to f64 -statement ok -create table t (c float) as values (1.0), (2.2), (1.0), (3.3), (null), (null), (3.3), (-2.0); - -query RT -select sum(distinct c), arrow_typeof(sum(distinct c)) from t; ----- -4.5 Float64 - -statement ok -drop table t; - -# aggregate sum distinct with decimal -statement ok -create table t (c decimal(35, 0)) as values (1), (2), (1), (3), (null), (null), (3), (-2); - -query RT -select sum(distinct c), arrow_typeof(sum(distinct c)) from t; ----- -4 Decimal128(38, 0) - -statement ok -drop table t; - -# query_sum_distinct - 2 different aggregate functions: avg and sum(distinct) -query RI -SELECT AVG(c1), SUM(DISTINCT c2) FROM test ----- -1.75 3 - -# query_sum_distinct - 2 sum(distinct) functions -query II -SELECT SUM(DISTINCT c1), SUM(DISTINCT c2) FROM test ----- -4 3 - -# # query_count_distinct -query I -SELECT COUNT(DISTINCT c1) FROM test ----- -3 - -# TODO: count_distinct_integers_aggregated_single_partition - -# TODO: count_distinct_integers_aggregated_multiple_partitions - -# TODO: aggregate_with_alias - -# test_approx_percentile_cont_decimal_support -query TI -SELECT c1, approx_percentile_cont(c2, cast(0.85 as decimal(10,2))) apc FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ----- -a 4 -b 5 -c 4 -d 4 -e 4 - -# array_agg_zero -query ? -SELECT ARRAY_AGG([]) ----- -[[]] - -# array_agg_one -query ? -SELECT ARRAY_AGG([1]) ----- -[[1]] - -# test array_agg with no row qualified -statement ok -create table t(a int, b float, c bigint) as values (1, 1.2, 2); - -# returns NULL, follows DuckDB's behaviour -query ? -select array_agg(a) from t where a > 2; ----- -NULL - -query ? -select array_agg(b) from t where b > 3.1; ----- -NULL - -query ? -select array_agg(c) from t where c > 3; ----- -NULL - -query ?I -select array_agg(c), count(1) from t where c > 3; ----- -NULL 0 - -# returns 0 rows if group by is applied, follows DuckDB's behaviour -query ? -select array_agg(a) from t where a > 3 group by a; ----- - -query ?I -select array_agg(a), count(1) from t where a > 3 group by a; ----- - -# returns NULL, follows DuckDB's behaviour -query ? -select array_agg(distinct a) from t where a > 3; ----- -NULL - -query ?I -select array_agg(distinct a), count(1) from t where a > 3; ----- -NULL 0 - -# returns 0 rows if group by is applied, follows DuckDB's behaviour -query ? -select array_agg(distinct a) from t where a > 3 group by a; ----- - -query ?I -select array_agg(distinct a), count(1) from t where a > 3 group by a; ----- - -# test order sensitive array agg -query ? -select array_agg(a order by a) from t where a > 3; ----- -NULL - -query ? -select array_agg(a order by a) from t where a > 3 group by a; ----- - -query ?I -select array_agg(a order by a), count(1) from t where a > 3 group by a; ----- - -statement ok -drop table t; - -# test with no values -statement ok -create table t(a int, b float, c bigint); - -query ? -select array_agg(a) from t; ----- -NULL - -query ? -select array_agg(b) from t; ----- -NULL - -query ? -select array_agg(c) from t; ----- -NULL - -query ?I -select array_agg(distinct a), count(1) from t; ----- -NULL 0 - -query ?I -select array_agg(distinct b), count(1) from t; ----- -NULL 0 - -query ?I -select array_agg(distinct b), count(1) from t; ----- -NULL 0 - -statement ok -drop table t; - - -# array_agg_i32 -statement ok -create table t (c1 int) as values (1), (2), (3), (4), (5); - -query ? -select array_agg(c1) from t; ----- -[1, 2, 3, 4, 5] - -statement ok -drop table t; - -# array_agg_nested -statement ok -create table t as values (make_array([1, 2, 3], [4, 5])), (make_array([6], [7, 8])), (make_array([9])); - -query ? -select array_agg(column1) from t; ----- -[[[1, 2, 3], [4, 5]], [[6], [7, 8]], [[9]]] - -statement ok -drop table t; - -# variance_single_value -query RRRR -select var(sq.column1), var_pop(sq.column1), stddev(sq.column1), stddev_pop(sq.column1) from (values (1.0)) as sq; ----- -NULL 0 NULL 0 - -# variance_two_values -query RRRR -select var(sq.column1), var_pop(sq.column1), stddev(sq.column1), stddev_pop(sq.column1) from (values (1.0), (3.0)) as sq; ----- -2 1 1.414213562373 1 - - - -# aggregates on empty tables -statement ok -CREATE TABLE empty (column1 bigint, column2 int); - -# no group by column -query IIRIIIII -SELECT - count(column1), -- counts should be zero, even for nulls - sum(column1), -- other aggregates should be null - avg(column1), - min(column1), - max(column1), - bit_and(column1), - bit_or(column1), - bit_xor(column1) -FROM empty ----- -0 NULL NULL NULL NULL NULL NULL NULL - -# Same query but with grouping (no groups, so no output) -query IIRIIIIII -SELECT - count(column1), - sum(column1), - avg(column1), - min(column1), - max(column1), - bit_and(column1), - bit_or(column1), - bit_xor(column1), - column2 -FROM empty -GROUP BY column2 -ORDER BY column2; ----- - - -statement ok -drop table empty - -# aggregates on all nulls -statement ok -CREATE TABLE the_nulls -AS VALUES - (null::bigint, 1), - (null::bigint, 1), - (null::bigint, 2); - -query II -select * from the_nulls ----- -NULL 1 -NULL 1 -NULL 2 - -# no group by column -query IIRIIIII -SELECT - count(column1), -- counts should be zero, even for nulls - sum(column1), -- other aggregates should be null - avg(column1), - min(column1), - max(column1), - bit_and(column1), - bit_or(column1), - bit_xor(column1) -FROM the_nulls ----- -0 NULL NULL NULL NULL NULL NULL NULL - -# Same query but with grouping -query IIRIIIIII -SELECT - count(column1), -- counts should be zero, even for nulls - sum(column1), -- other aggregates should be null - avg(column1), - min(column1), - max(column1), - bit_and(column1), - bit_or(column1), - bit_xor(column1), - column2 -FROM the_nulls -GROUP BY column2 -ORDER BY column2; ----- -0 NULL NULL NULL NULL NULL NULL NULL 1 -0 NULL NULL NULL NULL NULL NULL NULL 2 - - -statement ok -drop table the_nulls; - -statement ok -create table bit_aggregate_functions ( - c1 SMALLINT NOT NULL, - c2 SMALLINT NOT NULL, - c3 SMALLINT, - tag varchar -) -as values - (5, 10, 11, 'A'), - (33, 11, null, 'B'), - (9, 12, null, 'A'); - -# query_bit_and, query_bit_or, query_bit_xor -query IIIIIIIII -SELECT - bit_and(c1), - bit_and(c2), - bit_and(c3), - bit_or(c1), - bit_or(c2), - bit_or(c3), - bit_xor(c1), - bit_xor(c2), - bit_xor(c3) -FROM bit_aggregate_functions ----- -1 8 11 45 15 11 45 13 11 - -# query_bit_and, query_bit_or, query_bit_xor, with group -query IIIIIIIIIT -SELECT - bit_and(c1), - bit_and(c2), - bit_and(c3), - bit_or(c1), - bit_or(c2), - bit_or(c3), - bit_xor(c1), - bit_xor(c2), - bit_xor(c3), - tag -FROM bit_aggregate_functions -GROUP BY tag -ORDER BY tag ----- -1 8 11 13 14 11 12 6 11 A -33 11 NULL 33 11 NULL 33 11 NULL B - - -# bit_and_i32 -statement ok -create table t (c int) as values (4), (7), (15); - -query IT -Select bit_and(c), arrow_typeof(bit_and(c)) from t; ----- -4 Int32 - -statement ok -drop table t; - -# bit_and_i32_with_nulls -statement ok -create table t (c int) as values (1), (NULL), (3), (5); - -query IT -Select bit_and(c), arrow_typeof(bit_and(c)) from t; ----- -1 Int32 - -statement ok -drop table t; - -# bit_and_i32_all_nulls -statement ok -create table t (c int) as values (NULL), (NULL); - -query IT -Select bit_and(c), arrow_typeof(bit_and(c)) from t; ----- -NULL Int32 - -statement ok -drop table t; - -# bit_and_u32 -statement ok -create table t (c int unsigned) as values (4), (7), (15); - -query IT -Select bit_and(c), arrow_typeof(bit_and(c)) from t; ----- -4 UInt32 - -statement ok -drop table t; - -# bit_or_i32 -statement ok -create table t (c int) as values (4), (7), (15); - -query IT -Select bit_or(c), arrow_typeof(bit_or(c)) from t; ----- -15 Int32 - -statement ok -drop table t; - -# bit_or_i32_with_nulls -statement ok -create table t (c int) as values (1), (NULL), (3), (5); - -query IT -Select bit_or(c), arrow_typeof(bit_or(c)) from t; ----- -7 Int32 - -statement ok -drop table t; - -#bit_or_i32_all_nulls -statement ok -create table t (c int) as values (NULL), (NULL); - -query IT -Select bit_or(c), arrow_typeof(bit_or(c)) from t; ----- -NULL Int32 - -statement ok -drop table t; - - -#bit_or_u32 -statement ok -create table t (c int unsigned) as values (4), (7), (15); - -query IT -Select bit_or(c), arrow_typeof(bit_or(c)) from t; ----- -15 UInt32 - -statement ok -drop table t; - -#bit_xor_i32 -statement ok -create table t (c int) as values (4), (7), (4), (7), (15); - -query IT -Select bit_xor(c), arrow_typeof(bit_xor(c)) from t; ----- -15 Int32 - -statement ok -drop table t; - -# bit_xor_i32_with_nulls -statement ok -create table t (c int) as values (1), (1), (NULL), (3), (5); - -query IT -Select bit_xor(c), arrow_typeof(bit_xor(c)) from t; ----- -6 Int32 - -statement ok -drop table t; - -# bit_xor_i32_all_nulls -statement ok -create table t (c int) as values (NULL), (NULL); - -query IT -Select bit_xor(c), arrow_typeof(bit_xor(c)) from t; ----- -NULL Int32 - -statement ok -drop table t; - -# bit_xor_u32 -statement ok -create table t (c int unsigned) as values (4), (7), (4), (7), (15); - -query IT -Select bit_xor(c), arrow_typeof(bit_xor(c)) from t; ----- -15 UInt32 - -statement ok -drop table t; - -# bit_xor_distinct_i32 -statement ok -create table t (c int) as values (4), (7), (4), (7), (15); - -query IT -Select bit_xor(DISTINCT c), arrow_typeof(bit_xor(DISTINCT c)) from t; ----- -12 Int32 - -statement ok -drop table t; - -# bit_xor_distinct_i32_with_nulls -statement ok -create table t (c int) as values (1), (1), (NULL), (3), (5); - -query IT -Select bit_xor(DISTINCT c), arrow_typeof(bit_xor(DISTINCT c)) from t; ----- -7 Int32 - - -statement ok -drop table t; - -# bit_xor_distinct_i32_all_nulls -statement ok -create table t (c int ) as values (NULL), (NULL); - -query IT -Select bit_xor(DISTINCT c), arrow_typeof(bit_xor(DISTINCT c)) from t; ----- -NULL Int32 - - -statement ok -drop table t; - -# bit_xor_distinct_u32 -statement ok -create table t (c int unsigned) as values (4), (7), (4), (7), (15); - -query IT -Select bit_xor(DISTINCT c), arrow_typeof(bit_xor(DISTINCT c)) from t; ----- -12 UInt32 - -statement ok -drop table t; - -################# -# Min_Max Begin # -################# -# min_decimal, max_decimal -statement ok -CREATE TABLE decimals (value DECIMAL(10, 2)); - -statement ok -INSERT INTO decimals VALUES (123.0001), (124.00); - -query RR -SELECT MIN(value), MAX(value) FROM decimals; ----- -123 124 - -statement ok -DROP TABLE decimals; - -statement ok -CREATE TABLE decimals_batch (value DECIMAL(10, 0)); - -statement ok -INSERT INTO decimals_batch VALUES (1), (2), (3), (4), (5); - -query RR -SELECT MIN(value), MAX(value) FROM decimals_batch; ----- -1 5 - -statement ok -DROP TABLE decimals_batch; - -statement ok -CREATE TABLE decimals_empty (value DECIMAL(10, 0)); - -query RR -SELECT MIN(value), MAX(value) FROM decimals_empty; ----- -NULL NULL - -statement ok -DROP TABLE decimals_empty; - -# min_decimal_all_nulls, max_decimal_all_nulls -statement ok -CREATE TABLE decimals_all_nulls (value DECIMAL(10, 0)); - -statement ok -INSERT INTO decimals_all_nulls VALUES (NULL), (NULL), (NULL), (NULL), (NULL), (NULL); - -query RR -SELECT MIN(value), MAX(value) FROM decimals_all_nulls; ----- -NULL NULL - -statement ok -DROP TABLE decimals_all_nulls; - -# min_decimal_with_nulls, max_decimal_with_nulls -statement ok -CREATE TABLE decimals_with_nulls (value DECIMAL(10, 0)); - -statement ok -INSERT INTO decimals_with_nulls VALUES (1), (NULL), (3), (4), (5); - -query RR -SELECT MIN(value), MAX(value) FROM decimals_with_nulls; ----- -1 5 - -statement ok -DROP TABLE decimals_with_nulls; - -statement ok -CREATE TABLE decimals_error (value DECIMAL(10, 2)); - -statement ok -INSERT INTO decimals_error VALUES (123.00), (arrow_cast(124.001, 'Decimal128(10, 3)')); - -query RR -SELECT MIN(value), MAX(value) FROM decimals_error; ----- -123 124 - -statement ok -DROP TABLE decimals_error; - -statement ok -CREATE TABLE decimals_agg (value DECIMAL(10, 0)); - -statement ok -INSERT INTO decimals_agg VALUES (1), (2), (3), (4), (5); - -query RR -SELECT MIN(value), MAX(value) FROM decimals_agg; ----- -1 5 - -statement ok -DROP TABLE decimals_agg; - -# min_i32, max_i32 -statement ok -CREATE TABLE integers (value INT); - -statement ok -INSERT INTO integers VALUES (1), (2), (3), (4), (5); - -query II -SELECT MIN(value), MAX(value) FROM integers ----- -1 5 - -statement ok -DROP TABLE integers; - -# min_utf8, max_utf8 -statement ok -CREATE TABLE strings (value TEXT); - -statement ok -INSERT INTO strings VALUES ('d'), ('a'), ('c'), ('b'); - -query TT -SELECT MIN(value), MAX(value) FROM strings ----- -a d - -statement ok -DROP TABLE strings; - -# min_i32_with_nulls, max_i32_with_nulls -statement ok -CREATE TABLE integers_with_nulls (value INT); - -statement ok -INSERT INTO integers_with_nulls VALUES (1), (NULL), (3), (4), (5); - -query II -SELECT MIN(value), MAX(value) FROM integers_with_nulls ----- -1 5 - -statement ok -DROP TABLE integers_with_nulls; - -# min_i32_all_nulls, max_i32_all_nulls -statement ok -CREATE TABLE integers_all_nulls (value INT); - -query II -SELECT MIN(value), MAX(value) FROM integers_all_nulls ----- -NULL NULL - -statement ok -DROP TABLE integers_all_nulls; - -# min_u32, max_u32 -statement ok -CREATE TABLE uintegers (value INT UNSIGNED); - -statement ok -INSERT INTO uintegers VALUES (1), (2), (3), (4), (5); - -query II -SELECT MIN(value), MAX(value) FROM uintegers ----- -1 5 - -statement ok -DROP TABLE uintegers; - -# min_f32, max_f32 -statement ok -CREATE TABLE floats (value FLOAT); - -statement ok -INSERT INTO floats VALUES (1.0), (2.0), (3.0), (4.0), (5.0); - -query RR -SELECT MIN(value), MAX(value) FROM floats ----- -1 5 - -statement ok -DROP TABLE floats; - -# min_f64, max_f64 -statement ok -CREATE TABLE doubles (value DOUBLE); - -statement ok -INSERT INTO doubles VALUES (1.0), (2.0), (3.0), (4.0), (5.0); - -query RR -SELECT MIN(value), MAX(value) FROM doubles ----- -1 5 - -statement ok -DROP TABLE doubles; - -# min_date, max_date -statement ok -CREATE TABLE dates (value DATE); - -statement ok -INSERT INTO dates VALUES ('1970-01-02'), ('1970-01-03'), ('1970-01-04'), ('1970-01-05'), ('1970-01-06'); - -query DD -SELECT MIN(value), MAX(value) FROM dates ----- -1970-01-02 1970-01-06 - -statement ok -DROP TABLE dates; - -# min_seconds, max_seconds -statement ok -CREATE TABLE times (value TIME); - -statement ok -INSERT INTO times VALUES ('00:00:01'), ('00:00:02'), ('00:00:03'), ('00:00:04'), ('00:00:05'); - -query DD -SELECT MIN(value), MAX(value) FROM times ----- -00:00:01 00:00:05 - -statement ok -DROP TABLE times; - -# min_milliseconds, max_milliseconds -statement ok -CREATE TABLE time32millisecond (value TIME); - -statement ok -INSERT INTO time32millisecond VALUES ('00:00:00.001'), ('00:00:00.002'), ('00:00:00.003'), ('00:00:00.004'), ('00:00:00.005'); - -query DD -SELECT MIN(value), MAX(value) FROM time32millisecond ----- -00:00:00.001 00:00:00.005 - -statement ok -DROP TABLE time32millisecond; - -# min_microseconds, max_microseconds -statement ok -CREATE TABLE time64microsecond (value TIME); - -statement ok -INSERT INTO time64microsecond VALUES ('00:00:00.000001'), ('00:00:00.000002'), ('00:00:00.000003'), ('00:00:00.000004'), ('00:00:00.000005'); - -query DD -SELECT MIN(value), MAX(value) FROM time64microsecond ----- -00:00:00.000001 00:00:00.000005 - -statement ok -DROP TABLE time64microsecond; - -# min_nanoseconds, max_nanoseconds -statement ok -CREATE TABLE time64nanosecond (value TIME); - -statement ok -INSERT INTO time64nanosecond VALUES ('00:00:00.000000001'), ('00:00:00.000000002'), ('00:00:00.000000003'), ('00:00:00.000000004'), ('00:00:00.000000005'); - -query DD -SELECT MIN(value), MAX(value) FROM time64nanosecond ----- -00:00:00.000000001 00:00:00.000000005 - -statement ok -DROP TABLE time64nanosecond; - -# min_timestamp, max_timestamp -statement ok -CREATE TABLE timestampmicrosecond (value TIMESTAMP); - -statement ok -INSERT INTO timestampmicrosecond VALUES ('1970-01-01 00:00:00.000001'), ('1970-01-01 00:00:00.000002'), ('1970-01-01 00:00:00.000003'), ('1970-01-01 00:00:00.000004'), ('1970-01-01 00:00:00.000005'); - -query PP -SELECT MIN(value), MAX(value) FROM timestampmicrosecond ----- -1970-01-01T00:00:00.000001 1970-01-01T00:00:00.000005 - -statement ok -DROP TABLE timestampmicrosecond; - -# max_bool -statement ok -CREATE TABLE max_bool (value BOOLEAN); - -statement ok -INSERT INTO max_bool VALUES (false), (false); - -query B -SELECT MAX(value) FROM max_bool ----- -false - -statement ok -DROP TABLE max_bool; - -statement ok -CREATE TABLE max_bool (value BOOLEAN); - -statement ok -INSERT INTO max_bool VALUES (true), (true); - -query B -SELECT MAX(value) FROM max_bool ----- -true - -statement ok -DROP TABLE max_bool; - -statement ok -CREATE TABLE max_bool (value BOOLEAN); - -statement ok -INSERT INTO max_bool VALUES (false), (true), (false); - -query B -SELECT MAX(value) FROM max_bool ----- -true - -statement ok -DROP TABLE max_bool; - -statement ok -CREATE TABLE max_bool (value BOOLEAN); - -statement ok -INSERT INTO max_bool VALUES (true), (false), (true); - -query B -SELECT MAX(value) FROM max_bool ----- -true - -statement ok -DROP TABLE max_bool; - -# min_bool -statement ok -CREATE TABLE min_bool (value BOOLEAN); - -statement ok -INSERT INTO min_bool VALUES (false), (false); - -query B -SELECT MIN(value) FROM min_bool ----- -false - -statement ok -DROP TABLE min_bool; - -statement ok -CREATE TABLE min_bool (value BOOLEAN); - -statement ok -INSERT INTO min_bool VALUES (true), (true); - -query B -SELECT MIN(value) FROM min_bool ----- -true - -statement ok -DROP TABLE min_bool; - -statement ok -CREATE TABLE min_bool (value BOOLEAN); - -statement ok -INSERT INTO min_bool VALUES (false), (true), (false); - -query B -SELECT MIN(value) FROM min_bool ----- -false - -statement ok -DROP TABLE min_bool; - -statement ok -CREATE TABLE min_bool (value BOOLEAN); - -statement ok -INSERT INTO min_bool VALUES (true), (false), (true); - -query B -SELECT MIN(value) FROM min_bool ----- -false - -statement ok -DROP TABLE min_bool; - -################# -# Min_Max End # -################# - -statement ok -create table bool_aggregate_functions ( - c1 boolean not null, - c2 boolean not null, - c3 boolean not null, - c4 boolean not null, - c5 boolean, - c6 boolean, - c7 boolean, - c8 boolean -) -as values - (true, true, false, false, true, true, null, null), - (true, false, true, false, false, null, false, null), - (true, true, false, false, null, true, false, null); - -# query_bool_and -query BBBBBBBB -SELECT bool_and(c1), bool_and(c2), bool_and(c3), bool_and(c4), bool_and(c5), bool_and(c6), bool_and(c7), bool_and(c8) FROM bool_aggregate_functions ----- -true false false false false true false NULL - -# query_bool_and_distinct -query BBBBBBBB -SELECT bool_and(distinct c1), bool_and(distinct c2), bool_and(distinct c3), bool_and(distinct c4), bool_and(distinct c5), bool_and(distinct c6), bool_and(distinct c7), bool_and(distinct c8) FROM bool_aggregate_functions ----- -true false false false false true false NULL - -# query_bool_or -query BBBBBBBB -SELECT bool_or(c1), bool_or(c2), bool_or(c3), bool_or(c4), bool_or(c5), bool_or(c6), bool_or(c7), bool_or(c8) FROM bool_aggregate_functions ----- -true true true false true true false NULL - -# query_bool_or_distinct -query BBBBBBBB -SELECT bool_or(distinct c1), bool_or(distinct c2), bool_or(distinct c3), bool_or(distinct c4), bool_or(distinct c5), bool_or(distinct c6), bool_or(distinct c7), bool_or(distinct c8) FROM bool_aggregate_functions ----- -true true true false true true false NULL - -# All supported timestamp types - -# "nanos" --> TimestampNanosecondArray -# "micros" --> TimestampMicrosecondArray -# "millis" --> TimestampMillisecondArray -# "secs" --> TimestampSecondArray -# "names" --> StringArray - -statement ok -create table t_source -as values - ('2018-11-13T17:11:10.011375885995', 'Row 0', 'X'), - ('2011-12-13T11:13:10.12345', 'Row 1', 'X'), - (null, 'Row 2', 'Y'), - ('2021-01-01T05:11:10.432', 'Row 3', 'Y'); - -statement ok -create table t as -select - arrow_cast(column1, 'Timestamp(Nanosecond, None)') as nanos, - arrow_cast(column1, 'Timestamp(Microsecond, None)') as micros, - arrow_cast(column1, 'Timestamp(Millisecond, None)') as millis, - arrow_cast(column1, 'Timestamp(Second, None)') as secs, - arrow_cast(column1, 'Timestamp(Nanosecond, Some("UTC"))') as nanos_utc, - arrow_cast(column1, 'Timestamp(Microsecond, Some("UTC"))') as micros_utc, - arrow_cast(column1, 'Timestamp(Millisecond, Some("UTC"))') as millis_utc, - arrow_cast(column1, 'Timestamp(Second, Some("UTC"))') as secs_utc, - column2 as names, - column3 as tag -from t_source; - -# Demonstrate the contents -query PPPPPPPPTT -select * from t; ----- -2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 X -2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 X -NULL NULL NULL NULL NULL NULL NULL NULL Row 2 Y -2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 Y - - -# aggregate_timestamps_sum -query error -SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t; - -query error -SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag; - -# aggregate_timestamps_count -query IIII -SELECT count(nanos), count(micros), count(millis), count(secs) FROM t; ----- -3 3 3 3 - -query TIIII -SELECT tag, count(nanos), count(micros), count(millis), count(secs) FROM t GROUP BY tag ORDER BY tag; ----- -X 2 2 2 2 -Y 1 1 1 1 - -# aggregate_timestamps_min -query PPPP -SELECT min(nanos), min(micros), min(millis), min(secs) FROM t; ----- -2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 - -query TPPPP -SELECT tag, min(nanos), min(micros), min(millis), min(secs) FROM t GROUP BY tag ORDER BY tag; ----- -X 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 -Y 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 - -# aggregate_timestamps_max -query PPPP -SELECT max(nanos), max(micros), max(millis), max(secs) FROM t; ----- -2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 - -query TPPPP -SELECT tag, max(nanos), max(micros), max(millis), max(secs) FROM t GROUP BY tag ORDER BY tag ----- -X 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 -Y 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 - -# aggregate_timestamps_count_distinct_with_tz -query IIII -SELECT count(DISTINCT nanos_utc), count(DISTINCT micros_utc), count(DISTINCT millis_utc), count(DISTINCT secs_utc) FROM t; ----- -3 3 3 3 - -query TIIII -SELECT tag, count(DISTINCT nanos_utc), count(DISTINCT micros_utc), count(DISTINCT millis_utc), count(DISTINCT secs_utc) FROM t GROUP BY tag ORDER BY tag; ----- -X 2 2 2 2 -Y 1 1 1 1 - -# aggregate_timestamps_avg -query error -SELECT avg(nanos), avg(micros), avg(millis), avg(secs) FROM t - -query error -SELECT tag, avg(nanos), avg(micros), avg(millis), avg(secs) FROM t GROUP BY tag ORDER BY tag; - -# aggregate_duration_array_agg -query T? -SELECT tag, array_agg(millis - arrow_cast(secs, 'Timestamp(Millisecond, None)')) FROM t GROUP BY tag ORDER BY tag; ----- -X [0 days 0 hours 0 mins 0.011 secs, 0 days 0 hours 0 mins 0.123 secs] -Y [, 0 days 0 hours 0 mins 0.432 secs] - -statement ok -drop table t_source; - -statement ok -drop table t; - - -# All supported Date tpes - -# "date32" --> Date32Array -# "date64" --> Date64Array -# "names" --> StringArray - -statement ok -create table t_source -as values - ('2018-11-13', 'Row 0', 'X'), - ('2011-12-13', 'Row 1', 'X'), - (null, 'Row 2', 'Y'), - ('2021-01-01', 'Row 3', 'Y'); - -statement ok -create table t as -select - arrow_cast(column1, 'Date32') as date32, - -- Workaround https://github.com/apache/arrow-rs/issues/4512 is fixed, can use this - -- arrow_cast(column1, 'Date64') as date64, - arrow_cast(arrow_cast(column1, 'Date32'), 'Date64') as date64, - column2 as names, - column3 as tag -from t_source; - -# Demonstrate the contents -query DDTT -select * from t; ----- -2018-11-13 2018-11-13T00:00:00 Row 0 X -2011-12-13 2011-12-13T00:00:00 Row 1 X -NULL NULL Row 2 Y -2021-01-01 2021-01-01T00:00:00 Row 3 Y - - -# aggregate_timestamps_sum -query error -SELECT sum(date32), sum(date64) FROM t; - -query error -SELECT tag, sum(date32), sum(date64) FROM t GROUP BY tag ORDER BY tag; - -# aggregate_timestamps_count -query II -SELECT count(date32), count(date64) FROM t; ----- -3 3 - -query TII -SELECT tag, count(date32), count(date64) FROM t GROUP BY tag ORDER BY tag; ----- -X 2 2 -Y 1 1 - -# aggregate_timestamps_min -query DD -SELECT min(date32), min(date64) FROM t; ----- -2011-12-13 2011-12-13T00:00:00 - -query TDD -SELECT tag, min(date32), min(date64) FROM t GROUP BY tag ORDER BY tag; ----- -X 2011-12-13 2011-12-13T00:00:00 -Y 2021-01-01 2021-01-01T00:00:00 - -# aggregate_timestamps_max -query DD -SELECT max(date32), max(date64) FROM t; ----- -2021-01-01 2021-01-01T00:00:00 - -query TDD -SELECT tag, max(date32), max(date64) FROM t GROUP BY tag ORDER BY tag ----- -X 2018-11-13 2018-11-13T00:00:00 -Y 2021-01-01 2021-01-01T00:00:00 - - -# aggregate_timestamps_avg -query error -SELECT avg(date32), avg(date64) FROM t - -query error -SELECT tag, avg(date32), avg(date64) FROM t GROUP BY tag ORDER BY tag; - - -statement ok -drop table t_source; - -statement ok -drop table t; - - -# All supported time types - -# Columns are named: -# "nanos" --> Time64NanosecondArray -# "micros" --> Time64MicrosecondArray -# "millis" --> Time32MillisecondArray -# "secs" --> Time32SecondArray -# "names" --> StringArray - -statement ok -create table t_source -as values - ('18:06:30.243620451', 'Row 0', 'A'), - ('20:08:28.161121654', 'Row 1', 'A'), - ('19:11:04.156423842', 'Row 2', 'B'), - ('21:06:28.247821084', 'Row 3', 'B'); - - -statement ok -create table t as -select - arrow_cast(column1, 'Time64(Nanosecond)') as nanos, - arrow_cast(column1, 'Time64(Microsecond)') as micros, - arrow_cast(column1, 'Time32(Millisecond)') as millis, - arrow_cast(column1, 'Time32(Second)') as secs, - column2 as names, - column3 as tag -from t_source; - -# Demonstrate the contents -query DDDDTT -select * from t; ----- -18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 Row 0 A -20:08:28.161121654 20:08:28.161121 20:08:28.161 20:08:28 Row 1 A -19:11:04.156423842 19:11:04.156423 19:11:04.156 19:11:04 Row 2 B -21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 Row 3 B - -# aggregate_times_sum -query error -SELECT sum(nanos), sum(micros), sum(millis), sum(secs) FROM t - -query error -SELECT tag, sum(nanos), sum(micros), sum(millis), sum(secs) FROM t GROUP BY tag ORDER BY tag - -# aggregate_times_count -query IIII -SELECT count(nanos), count(micros), count(millis), count(secs) FROM t ----- -4 4 4 4 - -query TIIII -SELECT tag, count(nanos), count(micros), count(millis), count(secs) FROM t GROUP BY tag ORDER BY tag ----- -A 2 2 2 2 -B 2 2 2 2 - - -# aggregate_times_min -query DDDD -SELECT min(nanos), min(micros), min(millis), min(secs) FROM t ----- -18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 - -query TDDDD -SELECT tag, min(nanos), min(micros), min(millis), min(secs) FROM t GROUP BY tag ORDER BY tag ----- -A 18:06:30.243620451 18:06:30.243620 18:06:30.243 18:06:30 -B 19:11:04.156423842 19:11:04.156423 19:11:04.156 19:11:04 - -# aggregate_times_max -query DDDD -SELECT max(nanos), max(micros), max(millis), max(secs) FROM t ----- -21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 - -query TDDDD -SELECT tag, max(nanos), max(micros), max(millis), max(secs) FROM t GROUP BY tag ORDER BY tag ----- -A 20:08:28.161121654 20:08:28.161121 20:08:28.161 20:08:28 -B 21:06:28.247821084 21:06:28.247821 21:06:28.247 21:06:28 - - -# aggregate_times_avg -query error -SELECT avg(nanos), avg(micros), avg(millis), avg(secs) FROM t - -query error -SELECT tag, avg(nanos), avg(micros), avg(millis), avg(secs) FROM t GROUP BY tag ORDER BY tag; - -statement ok -drop table t_source; - -statement ok -drop table t; - - -# aggregates on strings -statement ok -create table t_source -as values - ('Foo', 1), - ('Bar', 2), - (null, 2), - ('Baz', 1); - -statement ok -create table t as -select - arrow_cast(column1, 'Utf8') as utf8, - arrow_cast(column1, 'LargeUtf8') as largeutf8, - column2 as tag -from t_source; - -# No groupy -query TTITTI -SELECT - min(utf8), - max(utf8), - count(utf8), - min(largeutf8), - max(largeutf8), - count(largeutf8) -FROM t ----- -Bar Foo 3 Bar Foo 3 - - -# with groupby -query TTITTI -SELECT - min(utf8), - max(utf8), - count(utf8), - min(largeutf8), - max(largeutf8), - count(largeutf8) -FROM t -GROUP BY tag -ORDER BY tag ----- -Baz Foo 2 Baz Foo 2 -Bar Bar 1 Bar Bar 1 - - -statement ok -drop table t_source; - -statement ok -drop table t; - - -# aggregates on binary -statement ok -create table t_source -as values - ('Foo', 1), - ('Bar', 2), - (null, 2), - ('Baz', 1); - -statement ok -create table t as -select - arrow_cast(column1, 'Binary') as binary, - arrow_cast(column1, 'LargeBinary') as largebinary, - column2 as tag -from t_source; - -# No groupy -query ??I??I -SELECT - min(binary), - max(binary), - count(binary), - min(largebinary), - max(largebinary), - count(largebinary) -FROM t ----- -426172 466f6f 3 426172 466f6f 3 - -# with groupby -query ??I??I -SELECT - min(binary), - max(binary), - count(binary), - min(largebinary), - max(largebinary), - count(largebinary) -FROM t -GROUP BY tag -ORDER BY tag ----- -42617a 466f6f 2 42617a 466f6f 2 -426172 426172 1 426172 426172 1 - - - -statement ok -drop table t_source; - -statement ok -drop table t; - - -query I -select median(a) from (select 1 as a where 1=0); ----- -NULL - -query I -select approx_median(a) from (select 1 as a where 1=0); ----- -NULL - -# aggregate_decimal_sum -query RT -select sum(c1), arrow_typeof(sum(c1)) from d_table; ----- -100 Decimal128(20, 3) - -# aggregate sum with decimal -statement ok -create table t (c decimal(35, 3)) as values (10), (null), (20); - -query RT -select sum(c), arrow_typeof(sum(c)) from t; ----- -30 Decimal128(38, 3) - -statement ok -drop table t; - -# aggregate sum with i32, sum coerced result to i64 -statement ok -create table t (c int) as values (1), (-1), (10), (null), (-11); - -query IT -select sum(c), arrow_typeof(sum(c)) from t; ----- --1 Int64 - -statement ok -drop table t; - -# aggregate sum with all nulls -statement ok -create table t (c1 decimal(10, 0), c2 int) as values (null, null), (null, null), (null, null); - -query RTIT -select - sum(c1), arrow_typeof(sum(c1)), - sum(c2), arrow_typeof(sum(c2)) -from t; ----- -NULL Decimal128(20, 0) NULL Int64 - -statement ok -drop table t; - -# aggregate sum with u32, sum coerced result to u64 -statement ok -create table t (c int unsigned) as values (1), (0), (10), (null), (4); - -query IT -select sum(c), arrow_typeof(sum(c)) from t; ----- -15 UInt64 - -statement ok -drop table t; - -# aggregate sum with f32, sum coerced result to f64 -statement ok -create table t (c float) as values (1.2), (0.2), (-1.2), (null), (-1.0); - -query RT -select sum(c), arrow_typeof(sum(c)) from t; ----- --0.79999999702 Float64 - -statement ok -drop table t; - -# aggregate sum with f64 -statement ok -create table t (c double) as values (1.2), (0.2), (-1.2), (null), (-1.0); - -query RT -select sum(c), arrow_typeof(sum(c)) from t; ----- --0.8 Float64 - -statement ok -drop table t; - -query TRT -select c2, sum(c1), arrow_typeof(sum(c1)) from d_table GROUP BY c2 ORDER BY c2; ----- -A 1100.045 Decimal128(20, 3) -B -1000.045 Decimal128(20, 3) - - -# aggregate_decimal_avg -query RT -select avg(c1), arrow_typeof(avg(c1)) from d_table ----- -5 Decimal128(14, 7) - -query TRT -select c2, avg(c1), arrow_typeof(avg(c1)) from d_table GROUP BY c2 ORDER BY c2 ----- -A 110.0045 Decimal128(14, 7) -B -100.0045 Decimal128(14, 7) - -# aggregate_decimal_count_distinct -query I -select count(DISTINCT cast(c1 AS DECIMAL(10, 2))) from d_table ----- -4 - -query TI -select c2, count(DISTINCT cast(c1 AS DECIMAL(10, 2))) from d_table GROUP BY c2 ORDER BY c2 ----- -A 2 -B 2 - -# Use PostgresSQL dialect -statement ok -set datafusion.sql_parser.dialect = 'Postgres'; - -# Creating the table -statement ok -CREATE TABLE test_table (c1 INT, c2 INT, c3 INT) - -# Inserting data -statement ok -INSERT INTO test_table VALUES - (1, 10, 50), - (1, 20, 60), - (2, 10, 70), - (2, 20, 80), - (3, 10, NULL) - -# query_group_by_with_filter -query III rowsort -SELECT - c1, - SUM(c2) FILTER (WHERE c2 >= 20), - SUM(c2) FILTER (WHERE c2 < 1) -- no rows pass filter, so the output should be NULL -FROM test_table GROUP BY c1 ----- -1 20 NULL -2 20 NULL -3 NULL NULL - -# query_group_by_avg_with_filter -query IRR rowsort -SELECT - c1, - AVG(c2) FILTER (WHERE c2 >= 20), - AVG(c2) FILTER (WHERE c2 < 1) -- no rows pass filter, so output should be null -FROM test_table GROUP BY c1 ----- -1 20 NULL -2 20 NULL -3 NULL NULL - -# query_group_by_with_multiple_filters -query IIR rowsort -SELECT - c1, - SUM(c2) FILTER (WHERE c2 >= 20) AS sum_c2, - AVG(c3) FILTER (WHERE c3 <= 70) AS avg_c3 -FROM test_table GROUP BY c1 ----- -1 20 55 -2 20 70 -3 NULL NULL - -# query_group_by_distinct_with_filter -query II rowsort -SELECT - c1, - COUNT(DISTINCT c2) FILTER (WHERE c2 >= 20) AS distinct_c2_count -FROM test_table GROUP BY c1 ----- -1 1 -2 1 -3 0 - -# query_without_group_by_with_filter -query I rowsort -SELECT - SUM(c2) FILTER (WHERE c2 >= 20) AS sum_c2 -FROM test_table ----- -40 - -# count_without_group_by_with_filter -query I rowsort -SELECT - COUNT(c2) FILTER (WHERE c2 >= 20) AS count_c2 -FROM test_table ----- -2 - -# query_with_and_without_filter -query III rowsort -SELECT - c1, - SUM(c2) FILTER (WHERE c2 >= 20) as result, - SUM(c2) as result_no_filter -FROM test_table GROUP BY c1; ----- -1 20 30 -2 20 30 -3 NULL 10 - -# query_filter_on_different_column_than_aggregate -query I rowsort -select - sum(c1) FILTER (WHERE c2 < 30) -FROM test_table; ----- -9 - -# query_test_empty_filter -query I rowsort -SELECT - SUM(c2) FILTER (WHERE c2 >= 20000000) AS sum_c2 -FROM test_table; ----- -NULL - -# Creating the decimal table -statement ok -CREATE TABLE test_decimal_table (c1 INT, c2 DECIMAL(5, 2), c3 DECIMAL(5, 1), c4 DECIMAL(5, 1)) - -# Inserting data -statement ok -INSERT INTO test_decimal_table VALUES (1, 10.10, 100.1, NULL), (1, 20.20, 200.2, NULL), (2, 10.10, 700.1, NULL), (2, 20.20, 700.1, NULL), (3, 10.1, 100.1, NULL), (3, 10.1, NULL, NULL) - -# aggregate_decimal_with_group_by -query IIRRRRIIRR rowsort -select c1, count(c2), avg(c2), sum(c2), min(c2), max(c2), count(c3), count(c4), sum(c4), avg(c4) from test_decimal_table group by c1 ----- -1 2 15.15 30.3 10.1 20.2 2 0 NULL NULL -2 2 15.15 30.3 10.1 20.2 2 0 NULL NULL -3 2 10.1 20.2 10.1 10.1 1 0 NULL NULL - -# aggregate_decimal_with_group_by_decimal -query RIRRRRIR rowsort -select c3, count(c2), avg(c2), sum(c2), min(c2), max(c2), count(c4), sum(c4) from test_decimal_table group by c3 ----- -100.1 2 10.1 20.2 10.1 10.1 0 NULL -200.2 1 20.2 20.2 20.2 20.2 0 NULL -700.1 2 15.15 30.3 10.1 20.2 0 NULL -NULL 1 10.1 10.1 10.1 10.1 0 NULL - -# Restore the default dialect -statement ok -set datafusion.sql_parser.dialect = 'Generic'; - -## Multiple distinct aggregates and dictionaries -statement ok -create table dict_test as values (1, arrow_cast('foo', 'Dictionary(Int32, Utf8)')), (2, arrow_cast('bar', 'Dictionary(Int32, Utf8)')); - -query I? -select * from dict_test; ----- -1 foo -2 bar - -query II -select count(distinct column1), count(distinct column2) from dict_test group by column1; ----- -1 1 -1 1 - -statement ok -drop table dict_test; - - -# Prepare the table with dictionary values for testing -statement ok -CREATE TABLE value(x bigint) AS VALUES (1), (2), (3), (1), (3), (4), (5), (2); - -statement ok -CREATE TABLE value_dict AS SELECT arrow_cast(x, 'Dictionary(Int64, Int32)') AS x_dict FROM value; - -query ? -select x_dict from value_dict; ----- -1 -2 -3 -1 -3 -4 -5 -2 - -query I -select sum(x_dict) from value_dict; ----- -21 - -query R -select avg(x_dict) from value_dict; ----- -2.625 - -# distinct_average -query R -select avg(distinct x_dict) from value_dict; ----- -3 - -query error -select avg(x_dict), avg(distinct x_dict) from value_dict; - -query I -select min(x_dict) from value_dict; ----- -1 - -query I -select max(x_dict) from value_dict; ----- -5 - -query I -select sum(x_dict) from value_dict where x_dict > 3; ----- -9 - -query R -select avg(x_dict) from value_dict where x_dict > 3; ----- -4.5 - -query I -select min(x_dict) from value_dict where x_dict > 3; ----- -4 - -query I -select max(x_dict) from value_dict where x_dict > 3; ----- -5 - -query I -select sum(x_dict) from value_dict group by x_dict % 2 order by sum(x_dict); ----- -8 -13 - -query R -select avg(x_dict) from value_dict group by x_dict % 2 order by avg(x_dict); ----- -2.6 -2.666666666667 - -query I -select min(x_dict) from value_dict group by x_dict % 2 order by min(x_dict); ----- -1 -2 - -query I -select max(x_dict) from value_dict group by x_dict % 2 order by max(x_dict); ----- -4 -5 - -query T -select arrow_typeof(x_dict) from value_dict group by x_dict; ----- -Dictionary(Int64, Int32) -Dictionary(Int64, Int32) -Dictionary(Int64, Int32) -Dictionary(Int64, Int32) -Dictionary(Int64, Int32) - -statement ok -drop table value - -statement ok -drop table value_dict - - -# bool aggregation -statement ok -CREATE TABLE value_bool(x boolean, g int) AS VALUES (NULL, 0), (false, 0), (true, 0), (false, 1), (true, 2), (NULL, 3); - -query B -select min(x) from value_bool; ----- -false - -query B -select max(x) from value_bool; ----- -true - -query B -select min(x) from value_bool group by g order by g; ----- -false -false -true -NULL - -query B -select max(x) from value_bool group by g order by g; ----- -true -false -true -NULL - -# -# Push limit into distinct group-by aggregation tests -# - -# Make results deterministic -statement ok -set datafusion.optimizer.repartition_aggregations = false; - -# -query TT -EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; ----- -logical_plan -01)Limit: skip=0, fetch=5 -02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -04)------TableScan: aggregate_test_100 projection=[c3] -physical_plan -01)GlobalLimitExec: skip=0, fetch=5 -02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] -07)------------CoalescePartitionsExec -08)--------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true - -query I -SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; ----- -1 --40 -29 --85 --82 - -query TT -EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; ----- -logical_plan -01)Limit: skip=4, fetch=5 -02)--Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] -03)----TableScan: aggregate_test_100 projection=[c2, c3] -physical_plan -01)GlobalLimitExec: skip=4, fetch=5 -02)--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - -query II -SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; ----- -5 -82 -4 -111 -3 104 -3 13 -1 38 - -# The limit should only apply to the aggregations which group by c3 -query TT -EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; ----- -logical_plan -01)Limit: skip=0, fetch=4 -02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -03)----Projection: aggregate_test_100.c3 -04)------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] -05)--------Filter: aggregate_test_100.c3 >= Int16(10) AND aggregate_test_100.c3 <= Int16(20) -06)----------TableScan: aggregate_test_100 projection=[c2, c3], partial_filters=[aggregate_test_100.c3 >= Int16(10), aggregate_test_100.c3 <= Int16(20)] -physical_plan -01)GlobalLimitExec: skip=0, fetch=4 -02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[4] -03)----CoalescePartitionsExec -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=[] -08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - -query I -SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; ----- -13 -17 -12 -14 - -# An aggregate expression causes the limit to not be pushed to the aggregation -query TT -EXPLAIN SELECT max(c1), c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5; ----- -logical_plan -01)Projection: MAX(aggregate_test_100.c1), aggregate_test_100.c2, aggregate_test_100.c3 -02)--Limit: skip=0, fetch=5 -03)----Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[MAX(aggregate_test_100.c1)]] -04)------TableScan: aggregate_test_100 projection=[c1, c2, c3] -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)] -04)------CoalescePartitionsExec -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)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true - -# TODO(msirek): Extend checking in LimitedDistinctAggregation equal groupings to ignore the order of columns -# in the group-by column lists, so the limit could be pushed to the lowest AggregateExec in this case -query TT -EXPLAIN SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; ----- -logical_plan -01)Limit: skip=10, fetch=3 -02)--Aggregate: groupBy=[[aggregate_test_100.c3, aggregate_test_100.c2]], aggr=[[]] -03)----Projection: aggregate_test_100.c3, aggregate_test_100.c2 -04)------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] -05)--------TableScan: aggregate_test_100 projection=[c2, c3] -physical_plan -01)GlobalLimitExec: skip=10, fetch=3 -02)--AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] -03)----CoalescePartitionsExec -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=[] -08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - -query II -SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; ----- -57 1 --54 4 -112 3 - -query TT -EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; ----- -logical_plan -01)Limit: skip=0, fetch=3 -02)--Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] -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] -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 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - -query II -SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; ----- -NULL NULL -2 NULL -5 NULL - - -statement ok -set datafusion.optimizer.enable_distinct_aggregation_soft_limit = false; - -# The limit should not be pushed into the aggregations -query TT -EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; ----- -logical_plan -01)Limit: skip=0, fetch=5 -02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -04)------TableScan: aggregate_test_100 projection=[c3] -physical_plan -01)GlobalLimitExec: skip=0, fetch=5 -02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] -07)------------CoalescePartitionsExec -08)--------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true - -statement ok -set datafusion.optimizer.enable_distinct_aggregation_soft_limit = true; - -statement ok -set datafusion.optimizer.repartition_aggregations = true; - -# -# regr_*() tests -# - -# regr_*() invalid input -statement error -select regr_slope(); - -statement error -select regr_intercept(*); - -statement error -select regr_count(*) from aggregate_test_100; - -statement error -select regr_r2(1); - -statement error -select regr_avgx(1,2,3); - -statement error -select regr_avgy(1, 'foo'); - -statement error -select regr_sxx('foo', 1); - -statement error -select regr_syy('foo', 'bar'); - -statement error -select regr_sxy(NULL, 'bar'); - - - -# regr_*() NULL results -query RRRRRRRRR -select regr_slope(1,1), regr_intercept(1,1), regr_count(1,1), regr_r2(1,1), regr_avgx(1,1), regr_avgy(1,1), regr_sxx(1,1), regr_syy(1,1), regr_sxy(1,1); ----- -NULL NULL 1 NULL 1 1 0 0 0 - -query RRRRRRRRR -select regr_slope(1, NULL), regr_intercept(1, NULL), regr_count(1, NULL), regr_r2(1, NULL), regr_avgx(1, NULL), regr_avgy(1, NULL), regr_sxx(1, NULL), regr_syy(1, NULL), regr_sxy(1, NULL); ----- -NULL NULL 0 NULL NULL NULL NULL NULL NULL - -query RRRRRRRRR -select regr_slope(NULL, 1), regr_intercept(NULL, 1), regr_count(NULL, 1), regr_r2(NULL, 1), regr_avgx(NULL, 1), regr_avgy(NULL, 1), regr_sxx(NULL, 1), regr_syy(NULL, 1), regr_sxy(NULL, 1); ----- -NULL NULL 0 NULL NULL NULL NULL NULL NULL - -query RRRRRRRRR -select regr_slope(NULL, NULL), regr_intercept(NULL, NULL), regr_count(NULL, NULL), regr_r2(NULL, NULL), regr_avgx(NULL, NULL), regr_avgy(NULL, NULL), regr_sxx(NULL, NULL), regr_syy(NULL, NULL), regr_sxy(NULL, NULL); ----- -NULL NULL 0 NULL NULL NULL NULL NULL NULL - -query RRRRRRRRR -select regr_slope(column2, column1), regr_intercept(column2, column1), regr_count(column2, column1), regr_r2(column2, column1), regr_avgx(column2, column1), regr_avgy(column2, column1), regr_sxx(column2, column1), regr_syy(column2, column1), regr_sxy(column2, column1) from (values (1,2), (1,4), (1,6)); ----- -NULL NULL 3 NULL 1 4 0 8 0 - - - -# regr_*() basic tests -query RRRRRRRRR -select - regr_slope(column2, column1), - regr_intercept(column2, column1), - regr_count(column2, column1), - regr_r2(column2, column1), - regr_avgx(column2, column1), - regr_avgy(column2, column1), - regr_sxx(column2, column1), - regr_syy(column2, column1), - regr_sxy(column2, column1) -from (values (1,2), (2,4), (3,6)); ----- -2 0 3 1 2 4 2 8 4 - -query RRRRRRRRR -select - regr_slope(c12, c11), - regr_intercept(c12, c11), - regr_count(c12, c11), - regr_r2(c12, c11), - regr_avgx(c12, c11), - regr_avgy(c12, c11), - regr_sxx(c12, c11), - regr_syy(c12, c11), - regr_sxy(c12, c11) -from aggregate_test_100; ----- -0.051534002628 0.48427355347 100 0.001929150558 0.479274948239 0.508972509913 6.707779292571 9.234223721582 0.345678715695 - - - -# regr_*() functions ignore NULLs -query RRRRRRRRR -select - regr_slope(column2, column1), - regr_intercept(column2, column1), - regr_count(column2, column1), - regr_r2(column2, column1), - regr_avgx(column2, column1), - regr_avgy(column2, column1), - regr_sxx(column2, column1), - regr_syy(column2, column1), - regr_sxy(column2, column1) -from (values (1,NULL), (2,4), (3,6)); ----- -2 0 2 1 2.5 5 0.5 2 1 - -query RRRRRRRRR -select - regr_slope(column2, column1), - regr_intercept(column2, column1), - regr_count(column2, column1), - regr_r2(column2, column1), - regr_avgx(column2, column1), - regr_avgy(column2, column1), - regr_sxx(column2, column1), - regr_syy(column2, column1), - regr_sxy(column2, column1) -from (values (1,NULL), (NULL,4), (3,6)); ----- -NULL NULL 1 NULL 3 6 0 0 0 - -query RRRRRRRRR -select - regr_slope(column2, column1), - regr_intercept(column2, column1), - regr_count(column2, column1), - regr_r2(column2, column1), - regr_avgx(column2, column1), - regr_avgy(column2, column1), - regr_sxx(column2, column1), - regr_syy(column2, column1), - regr_sxy(column2, column1) -from (values (1,NULL), (NULL,4), (NULL,NULL)); ----- -NULL NULL 0 NULL NULL NULL NULL NULL NULL - -query TRRRRRRRRR rowsort -select - column3, - regr_slope(column2, column1), - regr_intercept(column2, column1), - regr_count(column2, column1), - regr_r2(column2, column1), - regr_avgx(column2, column1), - regr_avgy(column2, column1), - regr_sxx(column2, column1), - regr_syy(column2, column1), - regr_sxy(column2, column1) -from (values (1,2,'a'), (2,4,'a'), (1,3,'b'), (3,9,'b'), (1,10,'c'), (NULL,100,'c')) -group by column3; ----- -a 2 0 2 1 1.5 3 0.5 2 1 -b 3 0 2 1 2 6 2 18 6 -c NULL NULL 1 NULL 1 10 0 0 0 - - - -# regr_*() testing merge_batch() from RegrAccumulator's internal implementation -statement ok -set datafusion.execution.batch_size = 1; - -query RRRRRRRRR -select - regr_slope(c12, c11), - regr_intercept(c12, c11), - regr_count(c12, c11), - regr_r2(c12, c11), - regr_avgx(c12, c11), - regr_avgy(c12, c11), - regr_sxx(c12, c11), - regr_syy(c12, c11), - regr_sxy(c12, c11) -from aggregate_test_100; ----- -0.051534002628 0.48427355347 100 0.001929150558 0.479274948239 0.508972509913 6.707779292571 9.234223721582 0.345678715695 - -statement ok -set datafusion.execution.batch_size = 2; - -query RRRRRRRRR -select - regr_slope(c12, c11), - regr_intercept(c12, c11), - regr_count(c12, c11), - regr_r2(c12, c11), - regr_avgx(c12, c11), - regr_avgy(c12, c11), - regr_sxx(c12, c11), - regr_syy(c12, c11), - regr_sxy(c12, c11) -from aggregate_test_100; ----- -0.051534002628 0.48427355347 100 0.001929150558 0.479274948239 0.508972509913 6.707779292571 9.234223721582 0.345678715695 - -statement ok -set datafusion.execution.batch_size = 3; - -query RRRRRRRRR -select - regr_slope(c12, c11), - regr_intercept(c12, c11), - regr_count(c12, c11), - regr_r2(c12, c11), - regr_avgx(c12, c11), - regr_avgy(c12, c11), - regr_sxx(c12, c11), - regr_syy(c12, c11), - regr_sxy(c12, c11) -from aggregate_test_100; ----- -0.051534002628 0.48427355347 100 0.001929150558 0.479274948239 0.508972509913 6.707779292571 9.234223721582 0.345678715695 - -statement ok -set datafusion.execution.batch_size = 8192; - - - -# regr_*() testing retract_batch() from RegrAccumulator's internal implementation -query RRRRRRRRR -SELECT - regr_slope(column2, column1) OVER w AS slope, - regr_intercept(column2, column1) OVER w AS intercept, - regr_count(column2, column1) OVER w AS count, - regr_r2(column2, column1) OVER w AS r2, - regr_avgx(column2, column1) OVER w AS avgx, - regr_avgy(column2, column1) OVER w AS avgy, - regr_sxx(column2, column1) OVER w AS sxx, - regr_syy(column2, column1) OVER w AS syy, - regr_sxy(column2, column1) OVER w AS sxy -FROM (VALUES (1,2), (2,4), (3,6), (4,12), (5,15), (6,18)) AS t(column1, column2) -WINDOW w AS (ORDER BY column1 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW); ----- -NULL NULL 1 NULL 1 2 0 0 0 -2 0 2 1 1.5 3 0.5 2 1 -2 0 3 1 2 4 2 8 4 -4 -4.666666666667 3 0.923076923077 3 7.333333333333 2 34.666666666667 8 -4.5 -7 3 0.964285714286 4 11 2 42 9 -3 0 3 1 5 15 2 18 6 - -query RRRRRRRRR -SELECT - regr_slope(column2, column1) OVER w AS slope, - regr_intercept(column2, column1) OVER w AS intercept, - regr_count(column2, column1) OVER w AS count, - regr_r2(column2, column1) OVER w AS r2, - regr_avgx(column2, column1) OVER w AS avgx, - regr_avgy(column2, column1) OVER w AS avgy, - regr_sxx(column2, column1) OVER w AS sxx, - regr_syy(column2, column1) OVER w AS syy, - regr_sxy(column2, column1) OVER w AS sxy -FROM (VALUES (1,2), (2,4), (3,6), (3, NULL), (4, NULL), (5,15), (6,18), (7, 21)) AS t(column1, column2) -WINDOW w AS (ORDER BY column1 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW); ----- -NULL NULL 1 NULL 1 2 0 0 0 -2 0 2 1 1.5 3 0.5 2 1 -2 0 3 1 2 4 2 8 4 -2 0 2 1 2.5 5 0.5 2 1 -NULL NULL 1 NULL 3 6 0 0 0 -NULL NULL 1 NULL 5 15 0 0 0 -3 0 2 1 5.5 16.5 0.5 4.5 1.5 -3 0 3 1 6 18 2 18 6 - -statement error -SELECT STRING_AGG() - -statement error -SELECT STRING_AGG(1,2,3) - -statement error -SELECT STRING_AGG(STRING_AGG('a', ',')) - -query T -SELECT STRING_AGG('a', ',') ----- -a - -query TTTT -SELECT STRING_AGG('a',','), STRING_AGG('a', NULL), STRING_AGG(NULL, ','), STRING_AGG(NULL, NULL) ----- -a a NULL NULL - -query TT -select string_agg('', '|'), string_agg('a', ''); ----- -(empty) a - -query T -SELECT STRING_AGG(column1, '|') FROM (values (''), (null), ('')); ----- -| - -statement ok -CREATE TABLE strings(g INTEGER, x VARCHAR, y VARCHAR) - -query I -INSERT INTO strings VALUES (1,'a','/'), (1,'b','-'), (2,'i','/'), (2,NULL,'-'), (2,'j','+'), (3,'p','/'), (4,'x','/'), (4,'y','-'), (4,'z','+') ----- -9 - -query IT -SELECT g, STRING_AGG(x,'|') FROM strings GROUP BY g ORDER BY g ----- -1 a|b -2 i|j -3 p -4 x|y|z - -query T -SELECT STRING_AGG(x,',') FROM strings WHERE g > 100 ----- -NULL - -statement ok -drop table strings - -query T -WITH my_data as ( -SELECT 'text1'::varchar(1000) as my_column union all -SELECT 'text1'::varchar(1000) as my_column union all -SELECT 'text1'::varchar(1000) as my_column -) -SELECT string_agg(my_column,', ') as my_string_agg -FROM my_data ----- -text1, text1, text1 - -query T -WITH my_data as ( -SELECT 1 as dummy, 'text1'::varchar(1000) as my_column union all -SELECT 1 as dummy, 'text1'::varchar(1000) as my_column union all -SELECT 1 as dummy, 'text1'::varchar(1000) as my_column -) -SELECT string_agg(my_column,', ') as my_string_agg -FROM my_data -GROUP BY dummy ----- -text1, text1, text1 - -# Tests for aggregating with NaN values -statement ok -CREATE TABLE float_table ( - col_f32 FLOAT, - col_f32_nan FLOAT, - col_f64 DOUBLE, - col_f64_nan DOUBLE -) as VALUES -( -128.2, -128.2, -128.2, -128.2 ), -( 32768.3, arrow_cast('NAN','Float32'), 32768.3, 32768.3 ), -( 27.3, 27.3, 27.3, arrow_cast('NAN','Float64') ); - -# Test string_agg with largeutf8 -statement ok -create table string_agg_large_utf8 (c string) as values - (arrow_cast('a', 'LargeUtf8')), - (arrow_cast('b', 'LargeUtf8')), - (arrow_cast('c', 'LargeUtf8')) -; - -query T -SELECT STRING_AGG(c, ',') FROM string_agg_large_utf8; ----- -a,b,c - -statement ok -drop table string_agg_large_utf8; - -query RRRRI -select min(col_f32), max(col_f32), avg(col_f32), sum(col_f32), count(col_f32) from float_table; ----- --128.2 32768.3 10889.13359451294 32667.40078353882 3 - -query RRRRI -select min(col_f32_nan), max(col_f32_nan), avg(col_f32_nan), sum(col_f32_nan), count(col_f32_nan) from float_table; ----- --128.2 NaN NaN NaN 3 - -query RRRRI -select min(col_f64), max(col_f64), avg(col_f64), sum(col_f64), count(col_f64) from float_table; ----- --128.2 32768.3 10889.133333333333 32667.4 3 - -query RRRRI -select min(col_f64_nan), max(col_f64_nan), avg(col_f64_nan), sum(col_f64_nan), count(col_f64_nan) from float_table; ----- --128.2 NaN NaN NaN 3 - -statement ok -drop table float_table - - -# Queries with nested count(*) - -query I -select count(*) from (select count(*) from (select 1)); ----- -1 - -query I -select count(*) from (select count(*) a, count(*) b from (select 1)); ----- -1 - -# Distinct Count for string -# (test for the specialized implementation of distinct count for strings) - -# UTF8 string matters for string to &[u8] conversion, add it to prevent regression -statement ok -create table distinct_count_string_table as values - (1, 'a', 'longstringtest_a', '台灣'), - (2, 'b', 'longstringtest_b1', '日本'), - (2, 'b', 'longstringtest_b2', '中國'), - (3, 'c', 'longstringtest_c1', '美國'), - (3, 'c', 'longstringtest_c2', '歐洲'), - (3, 'c', 'longstringtest_c3', '韓國') -; - -# run through update_batch -query IIII -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_string_table; ----- -3 3 6 6 - -# run through merge_batch -query IIII rowsort -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_string_table group by column1; ----- -1 1 1 1 -1 1 2 2 -1 1 3 3 - - -# test with long strings as well -statement ok -create table distinct_count_long_string_table as -SELECT column1, - arrow_cast(column2, 'LargeUtf8') as column2, - arrow_cast(column3, 'LargeUtf8') as column3, - arrow_cast(column4, 'LargeUtf8') as column4 -FROM distinct_count_string_table; - -# run through update_batch -query IIII -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_long_string_table; ----- -3 3 6 6 - -# run through merge_batch -query IIII rowsort -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_long_string_table group by column1; ----- -1 1 1 1 -1 1 2 2 -1 1 3 3 - -statement ok -drop table distinct_count_long_string_table; - - -# test with binary strings as well -statement ok -create table distinct_count_binary_table as -SELECT column1, - arrow_cast(column2, 'Binary') as column2, - arrow_cast(column3, 'Binary') as column3, - arrow_cast(column4, 'Binary') as column4 -FROM distinct_count_string_table; - -# run through update_batch -query IIII -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_binary_table; ----- -3 3 6 6 - -# run through merge_batch -query IIII rowsort -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_binary_table group by column1; ----- -1 1 1 1 -1 1 2 2 -1 1 3 3 - -statement ok -drop table distinct_count_binary_table; - - -# test with large binary strings as well -statement ok -create table distinct_count_large_binary_table as -SELECT column1, - arrow_cast(column2, 'LargeBinary') as column2, - arrow_cast(column3, 'LargeBinary') as column3, - arrow_cast(column4, 'LargeBinary') as column4 -FROM distinct_count_string_table; - -# run through update_batch -query IIII -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_large_binary_table; ----- -3 3 6 6 - -# run through merge_batch -query IIII rowsort -select count(distinct column1), count(distinct column2), count(distinct column3), count(distinct column4) from distinct_count_large_binary_table group by column1; ----- -1 1 1 1 -1 1 2 2 -1 1 3 3 - -statement ok -drop table distinct_count_large_binary_table; - - - -## Cleanup from distinct count tests -statement ok -drop table distinct_count_string_table; - - - -# rule `aggregate_statistics` should not optimize MIN/MAX to wrong values on empty relation - -statement ok -CREATE TABLE empty(col0 INTEGER); - -query I -SELECT MIN(col0) FROM empty WHERE col0=1; ----- -NULL - -query I -SELECT MAX(col0) FROM empty WHERE col0=1; ----- -NULL - -query TT -EXPLAIN SELECT MIN(col0) FROM empty; ----- -logical_plan -01)Aggregate: groupBy=[[]], aggr=[[MIN(empty.col0)]] -02)--TableScan: empty projection=[col0] -physical_plan -01)ProjectionExec: expr=[NULL as MIN(empty.col0)] -02)--PlaceholderRowExec - -query TT -EXPLAIN SELECT MAX(col0) FROM empty; ----- -logical_plan -01)Aggregate: groupBy=[[]], aggr=[[MAX(empty.col0)]] -02)--TableScan: empty projection=[col0] -physical_plan -01)ProjectionExec: expr=[NULL as MAX(empty.col0)] -02)--PlaceholderRowExec - -statement ok -DROP TABLE empty; - -statement ok -CREATE TABLE t(col0 INTEGER) as VALUES(2); - -query I -SELECT MIN(col0) FROM t WHERE col0=1; ----- -NULL - -query I -SELECT MAX(col0) FROM t WHERE col0=1; ----- -NULL - -statement ok -DROP TABLE t; - - -# Test for the case when the column name is ambiguous -statement ok -CREATE TABLE t(a BIGINT) AS VALUES(1), (2), (3); - -# The column name referenced by GROUP-BY is ambiguous, prefer the column in base plan -query I -SELECT 0 as "t.a" FROM t GROUP BY t.a; ----- -0 -0 -0 - -# The column name referenced by HAVING is ambiguous, prefer the column in the base plan -query I -SELECT 0 AS "t.a" FROM t HAVING MAX(t.a) = 0; ----- - -# Test issue: https://github.com/apache/datafusion/issues/9161 -query I rowsort -SELECT CAST(a AS INT) FROM t GROUP BY t.a; ----- -1 -2 -3 - -statement ok -DROP TABLE t; - -# Test for ignore null in FIRST_VALUE -statement ok -CREATE TABLE t AS VALUES (null::bigint), (3), (4); - -query I -SELECT FIRST_VALUE(column1) FROM t; ----- -NULL - -query I -SELECT FIRST_VALUE(column1) RESPECT NULLS FROM t; ----- -NULL - -query I -SELECT FIRST_VALUE(column1) IGNORE NULLS FROM t; ----- -3 - -statement ok -DROP TABLE t; - -# Test for ignore null with ORDER BY in FIRST_VALUE -statement ok -CREATE TABLE t AS VALUES (3, 4), (4, 3), (null::bigint, 1), (null::bigint, 1); - -query I -SELECT column1 FROM t ORDER BY column2; ----- -NULL -NULL -4 -3 - -query I -SELECT FIRST_VALUE(column1 ORDER BY column2) FROM t; ----- -NULL - -query I -SELECT FIRST_VALUE(column1 ORDER BY column2) RESPECT NULLS FROM t; ----- -NULL - -query I -SELECT FIRST_VALUE(column1 ORDER BY column2) IGNORE NULLS FROM t; ----- -4 - -statement ok -DROP TABLE t; - -# Test for ignore null in LAST_VALUE -statement ok -CREATE TABLE t AS VALUES (3), (4), (null::bigint); - -query I -SELECT LAST_VALUE(column1) FROM t; ----- -NULL - -query I -SELECT LAST_VALUE(column1) RESPECT NULLS FROM t; ----- -NULL - -query I -SELECT LAST_VALUE(column1) IGNORE NULLS FROM t; ----- -4 - -statement ok -DROP TABLE t; - -# Test for ignore null with ORDER BY in LAST_VALUE -statement ok -CREATE TABLE t AS VALUES (3, 3), (4, 4), (null::bigint, 1), (null::bigint, 2); - -query I -SELECT column1 FROM t ORDER BY column2 DESC; ----- -4 -3 -NULL -NULL - -query I -SELECT LAST_VALUE(column1 ORDER BY column2 DESC) FROM t; ----- -NULL - -query I -SELECT LAST_VALUE(column1 ORDER BY column2 DESC) RESPECT NULLS FROM t; ----- -NULL - -query I -SELECT LAST_VALUE(column1 ORDER BY column2 DESC) IGNORE NULLS FROM t; ----- -3 - -statement ok -DROP TABLE t; - -# Test for CASE with NULL in aggregate function -statement ok -CREATE TABLE example(data double precision); - -statement ok -INSERT INTO example VALUES (1), (2), (NULL), (4); - -query RR -SELECT - sum(CASE WHEN data is NULL THEN NULL ELSE data+1 END) as then_null, - sum(CASE WHEN data is NULL THEN data+1 ELSE NULL END) as else_null -FROM example; ----- -10 NULL - -query R -SELECT - CASE data WHEN 1 THEN NULL WHEN 2 THEN 3.3 ELSE NULL END as case_null -FROM example; ----- -NULL -3.3 -NULL -NULL - -statement ok -drop table example; - -# Test Convert FirstLast optimizer rule -statement ok -CREATE EXTERNAL TABLE convert_first_last_table ( -c1 INT NOT NULL, -c2 INT NOT NULL, -c3 INT NOT NULL -) -STORED AS CSV -WITH ORDER (c1 ASC) -WITH ORDER (c2 DESC) -WITH ORDER (c3 ASC) -LOCATION '../core/tests/data/convert_first_last.csv' -OPTIONS ('format.has_header' 'true'); - -# test first to last, the result does not show difference, we need to check the conversion by `explain` -query TT -explain select first_value(c1 order by c3 desc) from convert_first_last_table; ----- -logical_plan -01)Aggregate: groupBy=[[]], aggr=[[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] -02)--TableScan: convert_first_last_table projection=[c1, c3] -physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]] -02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true - -# test last to first -query TT -explain select last_value(c1 order by c2 asc) from convert_first_last_table; ----- -logical_plan -01)Aggregate: groupBy=[[]], aggr=[[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] -02)--TableScan: convert_first_last_table projection=[c1, c2] -physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]] -02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true - -# test building plan with aggreagte sum - -statement ok -create table employee_csv(id int, first_name string, last_name varchar, state varchar, salary bigint) as values (1, 'jenson', 'huang', 'unemployed', 10); - -query TI -select state, sum(salary) total_salary from employee_csv group by state; ----- -unemployed 10 - -statement ok -set datafusion.explain.logical_plan_only = true; - -query TT -explain select state, sum(salary) as total_salary from employee_csv group by state; ----- -logical_plan -01)Projection: employee_csv.state, sum(employee_csv.salary) AS total_salary -02)--Aggregate: groupBy=[[employee_csv.state]], aggr=[[sum(employee_csv.salary)]] -03)----TableScan: employee_csv projection=[state, salary] - -# fail if there is duplicate name -query error DataFusion error: Schema error: Schema contains qualified field name employee_csv\.state and unqualified field name state which would be ambiguous -select state, sum(salary) as state from employee_csv group by state; - -statement ok -set datafusion.explain.logical_plan_only = false; - -statement ok -drop table employee_csv; diff --git a/datafusion/sqllogictest/test_files/aggregate_debug.slt b/datafusion/sqllogictest/test_files/aggregate_debug.slt new file mode 100644 index 0000000000000..387507aee1562 --- /dev/null +++ b/datafusion/sqllogictest/test_files/aggregate_debug.slt @@ -0,0 +1,28 @@ +# 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. + +# aggregate Interval(MonthDayNano) min/max +query T?? +select + arrow_typeof(min(column1)) +from values + (interval '1 month'), + (interval '2 months'), + (interval '2 month 15 days'), + (interval '-2 month') +---- +Interval(MonthDayNano)