From 336c15e759c334debb7834088af698ec6b03763a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 5 Aug 2024 06:51:10 -0400 Subject: [PATCH 01/34] Improve MSRV CI check to print out problems to log (#11789) * Improve MSRV CI check to print out problems to log * Add reproduction instructions * Use `verify` and add `output-format` option * Update .github/workflows/rust.yml Co-authored-by: Eduard Karacharov --------- Co-authored-by: Eduard Karacharov --- .github/workflows/rust.yml | 32 +++++++++++++++++++++++--------- 1 file changed, 23 insertions(+), 9 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index fd8c2d2090b9..00d31765e77d 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -566,18 +566,32 @@ jobs: - name: Check datafusion working-directory: datafusion/core run: | - # If you encounter an error with any of the commands below - # it means some crate in your dependency tree has a higher - # MSRV (Min Supported Rust Version) than the one specified - # in the `rust-version` key of `Cargo.toml`. Check your - # dependencies or update the version in `Cargo.toml` - cargo msrv verify + # If you encounter an error with any of the commands below it means + # your code or some crate in the dependency tree has a higher MSRV + # (Min Supported Rust Version) than the one specified in the + # `rust-version` key of `Cargo.toml`. + # + # To reproduce: + # 1. Install the version of Rust that is failing. Example: + # rustup install 1.76.0 + # 2. Run the command that failed with that version. Example: + # cargo +1.76.0 check -p datafusion + # + # To resolve, either: + # 1. Change your code to use older Rust features, + # 2. Revert dependency update + # 3. Update the MSRV version in `Cargo.toml` + # + # Please see the DataFusion Rust Version Compatibility Policy before + # updating Cargo.toml. You may have to update the code instead. + # https://github.com/apache/datafusion/blob/main/README.md#rust-version-compatibility-policy + cargo msrv --output-format json --log-target stdout verify - name: Check datafusion-substrait working-directory: datafusion/substrait - run: cargo msrv verify + run: cargo msrv --output-format json --log-target stdout verify - name: Check datafusion-proto working-directory: datafusion/proto - run: cargo msrv verify + run: cargo msrv --output-format json --log-target stdout verify - name: Check datafusion-cli working-directory: datafusion-cli - run: cargo msrv verify \ No newline at end of file + run: cargo msrv --output-format json --log-target stdout verify \ No newline at end of file From c340b6ab7a2875c6924f825782f0c5ea408aff3d Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Mon, 5 Aug 2024 13:55:18 +0300 Subject: [PATCH 02/34] Skipping partial aggregation when it is not helping for high cardinality aggregates (#11627) * rfc: optional skipping partial aggregation * benchmarks for convert_to_state * speeding up conversion to state * Fix MSRV error on 1.76.0 * Improve aggregatation documentation --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/config.rs | 9 + datafusion/expr/src/accumulator.rs | 105 +++++- datafusion/expr/src/groups_accumulator.rs | 66 +++- datafusion/expr/src/udaf.rs | 4 +- datafusion/functions-aggregate/Cargo.toml | 10 + .../functions-aggregate/benches/count.rs | 98 ++++++ datafusion/functions-aggregate/benches/sum.rs | 106 ++++++ datafusion/functions-aggregate/src/count.rs | 66 ++++ .../aggregate/groups_accumulator/prim_op.rs | 61 +++- .../physical-plan/src/aggregates/mod.rs | 203 ++++++++++- .../physical-plan/src/aggregates/row_hash.rs | 218 +++++++++++- .../test_files/aggregate_skip_partial.slt | 324 ++++++++++++++++++ .../test_files/information_schema.slt | 4 + docs/source/user-guide/configs.md | 2 + 14 files changed, 1261 insertions(+), 15 deletions(-) create mode 100644 datafusion/functions-aggregate/benches/count.rs create mode 100644 datafusion/functions-aggregate/benches/sum.rs create mode 100644 datafusion/sqllogictest/test_files/aggregate_skip_partial.slt diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 9f8aa1cbdcaa..b5204b343f05 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -324,6 +324,15 @@ config_namespace! { /// Should DataFusion keep the columns used for partition_by in the output RecordBatches pub keep_partition_by_columns: bool, default = false + + /// Aggregation ratio (number of distinct groups / number of input rows) + /// threshold for skipping partial aggregation. If the value is greater + /// then partial aggregation will skip aggregation for further input + pub skip_partial_aggregation_probe_ratio_threshold: f64, default = 0.8 + + /// Number of input rows partial aggregation partition should process, before + /// aggregation ratio check and trying to switch to skipping aggregation mode + pub skip_partial_aggregation_probe_rows_threshold: usize, default = 100_000 } } diff --git a/datafusion/expr/src/accumulator.rs b/datafusion/expr/src/accumulator.rs index f9af7850cb92..262646d8ba3a 100644 --- a/datafusion/expr/src/accumulator.rs +++ b/datafusion/expr/src/accumulator.rs @@ -94,7 +94,7 @@ pub trait Accumulator: Send + Sync + Debug { /// /// Intermediate state is used for "multi-phase" grouping in /// DataFusion, where an aggregate is computed in parallel with - /// multiple `Accumulator` instances, as illustrated below: + /// multiple `Accumulator` instances, as described below: /// /// # MultiPhase Grouping /// @@ -130,7 +130,7 @@ pub trait Accumulator: Send + Sync + Debug { /// `───────' `───────' /// ``` /// - /// The partial state is serialied as `Arrays` and then combined + /// The partial state is serialized as `Arrays` and then combined /// with other partial states from different instances of this /// Accumulator (that ran on different partitions, for example). /// @@ -147,6 +147,107 @@ pub trait Accumulator: Send + Sync + Debug { /// Note that [`ScalarValue::List`] can be used to pass multiple /// values if the number of intermediate values is not known at /// planning time (e.g. for `MEDIAN`) + /// + /// # Multi-phase repartitioned Grouping + /// + /// Many multi-phase grouping plans contain a Repartition operation + /// as well as shown below: + /// + /// ```text + /// ▲ ▲ + /// │ │ + /// │ │ + /// │ │ + /// │ │ + /// │ │ + /// ┌───────────────────────┐ ┌───────────────────────┐ 4. Each AggregateMode::Final + /// │GroupBy │ │GroupBy │ GroupBy has an entry for its + /// │(AggregateMode::Final) │ │(AggregateMode::Final) │ subset of groups (in this case + /// │ │ │ │ that means half the entries) + /// └───────────────────────┘ └───────────────────────┘ + /// ▲ ▲ + /// │ │ + /// └─────────────┬────────────┘ + /// │ + /// │ + /// │ + /// ┌─────────────────────────┐ 3. Repartitioning by hash(group + /// │ Repartition │ keys) ensures that each distinct + /// │ HASH(x) │ group key now appears in exactly + /// └─────────────────────────┘ one partition + /// ▲ + /// │ + /// ┌───────────────┴─────────────┐ + /// │ │ + /// │ │ + /// ┌─────────────────────────┐ ┌──────────────────────────┐ 2. Each AggregateMode::Partial + /// │ GroubyBy │ │ GroubyBy │ GroupBy has an entry for *all* + /// │(AggregateMode::Partial) │ │ (AggregateMode::Partial) │ the groups + /// └─────────────────────────┘ └──────────────────────────┘ + /// ▲ ▲ + /// │ ┌┘ + /// │ │ + /// .─────────. .─────────. + /// ,─' '─. ,─' '─. + /// ; Input : ; Input : 1. Since input data is + /// : Partition 0 ; : Partition 1 ; arbitrarily or RoundRobin + /// ╲ ╱ ╲ ╱ distributed, each partition + /// '─. ,─' '─. ,─' likely has all distinct + /// `───────' `───────' + /// ``` + /// + /// This structure is used so that the `AggregateMode::Partial` accumulators + /// reduces the cardinality of the input as soon as possible. Typically, + /// each partial accumulator sees all groups in the input as the group keys + /// are evenly distributed across the input. + /// + /// The final output is computed by repartitioning the result of + /// [`Self::state`] from each Partial aggregate and `hash(group keys)` so + /// that each distinct group key appears in exactly one of the + /// `AggregateMode::Final` GroupBy nodes. The output of the final nodes are + /// then unioned together to produce the overall final output. + /// + /// Here is an example that shows the distribution of groups in the + /// different phases + /// + /// ```text + /// ┌─────┐ ┌─────┐ + /// │ 1 │ │ 3 │ + /// ├─────┤ ├─────┤ + /// │ 2 │ │ 4 │ After repartitioning by + /// └─────┘ └─────┘ hash(group keys), each distinct + /// ┌─────┐ ┌─────┐ group key now appears in exactly + /// │ 1 │ │ 3 │ one partition + /// ├─────┤ ├─────┤ + /// │ 2 │ │ 4 │ + /// └─────┘ └─────┘ + /// + /// + /// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ + /// + /// ┌─────┐ ┌─────┐ + /// │ 2 │ │ 2 │ + /// ├─────┤ ├─────┤ + /// │ 1 │ │ 2 │ + /// ├─────┤ ├─────┤ + /// │ 3 │ │ 3 │ + /// ├─────┤ ├─────┤ + /// │ 4 │ │ 1 │ + /// └─────┘ └─────┘ Input data is arbitrarily or + /// ... ... RoundRobin distributed, each + /// ┌─────┐ ┌─────┐ partition likely has all + /// │ 1 │ │ 4 │ distinct group keys + /// ├─────┤ ├─────┤ + /// │ 4 │ │ 3 │ + /// ├─────┤ ├─────┤ + /// │ 1 │ │ 1 │ + /// ├─────┤ ├─────┤ + /// │ 4 │ │ 3 │ + /// └─────┘ └─────┘ + /// + /// group values group values + /// in partition 0 in partition 1 + /// ``` fn state(&mut self) -> Result>; /// Updates the accumulator's state from an `Array` containing one diff --git a/datafusion/expr/src/groups_accumulator.rs b/datafusion/expr/src/groups_accumulator.rs index 0d57c403bbe0..886bd8443e4d 100644 --- a/datafusion/expr/src/groups_accumulator.rs +++ b/datafusion/expr/src/groups_accumulator.rs @@ -18,7 +18,7 @@ //! Vectorized [`GroupsAccumulator`] use arrow_array::{ArrayRef, BooleanArray}; -use datafusion_common::Result; +use datafusion_common::{not_impl_err, Result}; /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy)] @@ -128,6 +128,9 @@ pub trait GroupsAccumulator: Send { /// Returns the intermediate aggregate state for this accumulator, /// used for multi-phase grouping, resetting its internal state. /// + /// See [`Accumulator::state`] for more information on multi-phase + /// aggregation. + /// /// For example, `AVG` might return two arrays: `SUM` and `COUNT` /// but the `MIN` aggregate would just return a single array. /// @@ -135,11 +138,13 @@ pub trait GroupsAccumulator: Send { /// single `StructArray` rather than multiple arrays. /// /// See [`Self::evaluate`] for details on the required output - /// order and `emit_to`. + /// order and `emit_to`. + /// + /// [`Accumulator::state`]: crate::Accumulator::state fn state(&mut self, emit_to: EmitTo) -> Result>; /// Merges intermediate state (the output from [`Self::state`]) - /// into this accumulator's values. + /// into this accumulator's current state. /// /// For some aggregates (such as `SUM`), `merge_batch` is the same /// as `update_batch`, but for some aggregates (such as `COUNT`, @@ -158,8 +163,59 @@ pub trait GroupsAccumulator: Send { total_num_groups: usize, ) -> Result<()>; + /// Converts an input batch directly the intermediate aggregate state. + /// + /// This is the equivalent of treating each input row as its own group. It + /// is invoked when the Partial phase of a multi-phase aggregation is not + /// reducing the cardinality enough to warrant spending more effort on + /// pre-aggregation (see `Background` section below), and switches to + /// passing intermediate state directly on to the next aggregation phase. + /// + /// Examples: + /// * `COUNT`: an array of 1s for each row in the input batch. + /// * `SUM/MIN/MAX`: the input values themselves. + /// + /// # Arguments + /// * `values`: the input arguments to the accumulator + /// * `opt_filter`: if present, any row where `opt_filter[i]` is false should be ignored + /// + /// # Background + /// + /// In a multi-phase aggregation (see [`Accumulator::state`]), the initial + /// Partial phase reduces the cardinality of the input data as soon as + /// possible in the plan. + /// + /// This strategy is very effective for queries with a small number of + /// groups, as most of the data is aggregated immediately and only a small + /// amount of data must be repartitioned (see [`Accumulator::state`] for + /// background) + /// + /// However, for queries with a large number of groups, the Partial phase + /// often does not reduce the cardinality enough to warrant the memory and + /// CPU cost of actually performing the aggregation. For such cases, the + /// HashAggregate operator will dynamically switch to passing intermediate + /// state directly to the next aggregation phase with minimal processing + /// using this method. + /// + /// [`Accumulator::state`]: crate::Accumulator::state + fn convert_to_state( + &self, + _values: &[ArrayRef], + _opt_filter: Option<&BooleanArray>, + ) -> Result> { + not_impl_err!("Input batch conversion to state not implemented") + } + + /// Returns `true` if [`Self::convert_to_state`] is implemented to support + /// intermediate aggregate state conversion. + fn supports_convert_to_state(&self) -> bool { + false + } + /// Amount of memory used to store the state of this accumulator, - /// in bytes. This function is called once per batch, so it should - /// be `O(n)` to compute, not `O(num_groups)` + /// in bytes. + /// + /// This function is called once per batch, so it should be `O(n)` to + /// compute, not `O(num_groups)` fn size(&self) -> usize; } diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index f5eeef6b53bb..3a292b2b49bf 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -351,6 +351,8 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// Return the fields used to store the intermediate state of this accumulator. /// + /// See [`Accumulator::state`] for background information. + /// /// args: [`StateFieldsArgs`] contains arguments passed to the /// aggregate function's accumulator. /// @@ -388,7 +390,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// # Notes /// /// Even if this function returns true, DataFusion will still use - /// `Self::accumulator` for certain queries, such as when this aggregate is + /// [`Self::accumulator`] for certain queries, such as when this aggregate is /// used as a window function or when there no GROUP BY columns in the /// query. fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 43ddd37cfb6f..4f2bd864832e 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -50,4 +50,14 @@ paste = "1.0.14" sqlparser = { workspace = true } [dev-dependencies] +arrow = { workspace = true, features = ["test_utils"] } +criterion = "0.5" rand = { workspace = true } + +[[bench]] +name = "count" +harness = false + +[[bench]] +name = "sum" +harness = false diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs new file mode 100644 index 000000000000..875112ca8d47 --- /dev/null +++ b/datafusion/functions-aggregate/benches/count.rs @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{ArrayRef, BooleanArray}; +use arrow::datatypes::Int32Type; +use arrow::util::bench_util::{create_boolean_array, create_primitive_array}; +use arrow_schema::{DataType, Field, Schema}; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_common::DFSchema; +use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; +use datafusion_functions_aggregate::count::Count; +use std::sync::Arc; + +fn prepare_accumulator() -> Box { + let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Int32, true)])); + let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); + let accumulator_args = AccumulatorArgs { + data_type: &DataType::Int64, + schema: &schema, + dfschema: &df_schema, + ignore_nulls: false, + sort_exprs: &[], + is_reversed: false, + name: "COUNT(f)", + is_distinct: false, + input_types: &[DataType::Int32], + input_exprs: &[datafusion_expr::col("f")], + }; + let count_fn = Count::new(); + + count_fn + .create_groups_accumulator(accumulator_args) + .unwrap() +} + +fn convert_to_state_bench( + c: &mut Criterion, + name: &str, + values: ArrayRef, + opt_filter: Option<&BooleanArray>, +) { + let accumulator = prepare_accumulator(); + c.bench_function(name, |b| { + b.iter(|| { + black_box( + accumulator + .convert_to_state(&[values.clone()], opt_filter) + .unwrap(), + ) + }) + }); +} + +fn count_benchmark(c: &mut Criterion) { + let values = Arc::new(create_primitive_array::(8192, 0.0)) as ArrayRef; + convert_to_state_bench(c, "count convert state no nulls, no filter", values, None); + + let values = Arc::new(create_primitive_array::(8192, 0.3)) as ArrayRef; + convert_to_state_bench(c, "count convert state 30% nulls, no filter", values, None); + + let values = Arc::new(create_primitive_array::(8192, 0.3)) as ArrayRef; + convert_to_state_bench(c, "count convert state 70% nulls, no filter", values, None); + + let values = Arc::new(create_primitive_array::(8192, 0.0)) as ArrayRef; + let filter = create_boolean_array(8192, 0.0, 0.5); + convert_to_state_bench( + c, + "count convert state no nulls, filter", + values, + Some(&filter), + ); + + let values = Arc::new(create_primitive_array::(8192, 0.3)) as ArrayRef; + let filter = create_boolean_array(8192, 0.0, 0.5); + convert_to_state_bench( + c, + "count convert state nulls, filter", + values, + Some(&filter), + ); +} + +criterion_group!(benches, count_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs new file mode 100644 index 000000000000..dfaa93cdeff7 --- /dev/null +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{ArrayRef, BooleanArray}; +use arrow::datatypes::Int64Type; +use arrow::util::bench_util::{create_boolean_array, create_primitive_array}; +use arrow_schema::{DataType, Field, Schema}; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_common::DFSchema; +use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; +use datafusion_functions_aggregate::sum::Sum; +use std::sync::Arc; + +fn prepare_accumulator(data_type: &DataType) -> Box { + let schema = Arc::new(Schema::new(vec![Field::new("f", data_type.clone(), true)])); + let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); + let accumulator_args = AccumulatorArgs { + data_type, + schema: &schema, + dfschema: &df_schema, + ignore_nulls: false, + sort_exprs: &[], + is_reversed: false, + name: "SUM(f)", + is_distinct: false, + input_types: &[data_type.clone()], + input_exprs: &[datafusion_expr::col("f")], + }; + let sum_fn = Sum::new(); + + sum_fn.create_groups_accumulator(accumulator_args).unwrap() +} + +fn convert_to_state_bench( + c: &mut Criterion, + name: &str, + values: ArrayRef, + opt_filter: Option<&BooleanArray>, +) { + let accumulator = prepare_accumulator(values.data_type()); + c.bench_function(name, |b| { + b.iter(|| { + black_box( + accumulator + .convert_to_state(&[values.clone()], opt_filter) + .unwrap(), + ) + }) + }); +} + +fn count_benchmark(c: &mut Criterion) { + let values = Arc::new(create_primitive_array::(8192, 0.0)) as ArrayRef; + convert_to_state_bench(c, "sum i64 convert state no nulls, no filter", values, None); + + let values = Arc::new(create_primitive_array::(8192, 0.3)) as ArrayRef; + convert_to_state_bench( + c, + "sum i64 convert state 30% nulls, no filter", + values, + None, + ); + + let values = Arc::new(create_primitive_array::(8192, 0.3)) as ArrayRef; + convert_to_state_bench( + c, + "sum i64 convert state 70% nulls, no filter", + values, + None, + ); + + let values = Arc::new(create_primitive_array::(8192, 0.0)) as ArrayRef; + let filter = create_boolean_array(8192, 0.0, 0.5); + convert_to_state_bench( + c, + "sum i64 convert state no nulls, filter", + values, + Some(&filter), + ); + + let values = Arc::new(create_primitive_array::(8192, 0.3)) as ArrayRef; + let filter = create_boolean_array(8192, 0.0, 0.5); + convert_to_state_bench( + c, + "sum i64 convert state nulls, filter", + values, + Some(&filter), + ); +} + +criterion_group!(benches, count_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 64eb7253f5c9..aea05442536e 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -23,6 +23,7 @@ use std::{fmt::Debug, sync::Arc}; use arrow::{ array::{ArrayRef, AsArray}, + compute, datatypes::{ DataType, Date32Type, Date64Type, Decimal128Type, Decimal256Type, Field, Float16Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, @@ -440,6 +441,71 @@ impl GroupsAccumulator for CountGroupsAccumulator { Ok(vec![Arc::new(counts) as ArrayRef]) } + /// Converts an input batch directly to a state batch + /// + /// The state of `COUNT` is always a single Int64Array: + /// * `1` (for non-null, non filtered values) + /// * `0` (for null values) + fn convert_to_state( + &self, + values: &[ArrayRef], + opt_filter: Option<&BooleanArray>, + ) -> Result> { + let values = &values[0]; + + let state_array = match (values.logical_nulls(), opt_filter) { + (None, None) => { + // In case there is no nulls in input and no filter, returning array of 1 + Arc::new(Int64Array::from_value(1, values.len())) + } + (Some(nulls), None) => { + // If there are any nulls in input values -- casting `nulls` (true for values, false for nulls) + // of input array to Int64 + let nulls = BooleanArray::new(nulls.into_inner(), None); + compute::cast(&nulls, &DataType::Int64)? + } + (None, Some(filter)) => { + // If there is only filter + // - applying filter null mask to filter values by bitand filter values and nulls buffers + // (using buffers guarantees absence of nulls in result) + // - casting result of bitand to Int64 array + let (filter_values, filter_nulls) = filter.clone().into_parts(); + + let state_buf = match filter_nulls { + Some(filter_nulls) => &filter_values & filter_nulls.inner(), + None => filter_values, + }; + + let boolean_state = BooleanArray::new(state_buf, None); + compute::cast(&boolean_state, &DataType::Int64)? + } + (Some(nulls), Some(filter)) => { + // For both input nulls and filter + // - applying filter null mask to filter values by bitand filter values and nulls buffers + // (using buffers guarantees absence of nulls in result) + // - applying values null mask to filter buffer by another bitand on filter result and + // nulls from input values + // - casting result to Int64 array + let (filter_values, filter_nulls) = filter.clone().into_parts(); + + let filter_buf = match filter_nulls { + Some(filter_nulls) => &filter_values & filter_nulls.inner(), + None => filter_values, + }; + let state_buf = &filter_buf & nulls.inner(); + + let boolean_state = BooleanArray::new(state_buf, None); + compute::cast(&boolean_state, &DataType::Int64)? + } + }; + + Ok(vec![state_array]) + } + + fn supports_convert_to_state(&self) -> bool { + true + } + fn size(&self) -> usize { self.counts.capacity() * std::mem::size_of::() } diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs index debb36852b22..8d69646bd422 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs @@ -18,9 +18,11 @@ use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; +use arrow::buffer::NullBuffer; +use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; -use datafusion_common::Result; +use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr::{EmitTo, GroupsAccumulator}; use super::accumulate::NullState; @@ -134,6 +136,63 @@ where self.update_batch(values, group_indices, opt_filter, total_num_groups) } + /// Converts an input batch directly to a state batch + /// + /// The state is: + /// - self.prim_fn for all non null, non filtered values + /// - null otherwise + /// + fn convert_to_state( + &self, + values: &[ArrayRef], + opt_filter: Option<&BooleanArray>, + ) -> Result> { + let values = values[0].as_primitive::().clone(); + + // Initializing state with starting values + let initial_state = + PrimitiveArray::::from_value(self.starting_value, values.len()); + + // Recalculating values in case there is filter + let values = match opt_filter { + None => values, + Some(filter) => { + let (filter_values, filter_nulls) = filter.clone().into_parts(); + // Calculating filter mask as a result of bitand of filter, and converting it to null buffer + let filter_bool = match filter_nulls { + Some(filter_nulls) => filter_nulls.inner() & &filter_values, + None => filter_values, + }; + let filter_nulls = NullBuffer::from(filter_bool); + + // Rebuilding input values with a new nulls mask, which is equal to + // the union of original nulls and filter mask + let (dt, values_buf, original_nulls) = values.clone().into_parts(); + let nulls_buf = + NullBuffer::union(original_nulls.as_ref(), Some(&filter_nulls)); + PrimitiveArray::::new(values_buf, nulls_buf).with_data_type(dt) + } + }; + + let state_values = compute::binary_mut(initial_state, &values, |mut x, y| { + (self.prim_fn)(&mut x, y); + x + }); + let state_values = state_values + .map_err(|_| { + internal_datafusion_err!( + "initial_values underlying buffer must not be shared" + ) + })? + .map_err(DataFusionError::from)?; + + Ok(vec![Arc::new(state_values)]) + } + + fn supports_convert_to_state(&self) -> bool { + true + } + fn size(&self) -> usize { self.values.capacity() * std::mem::size_of::() + self.null_state.size() } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 43f9f98283bb..8941418c12e1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -56,11 +56,20 @@ mod topk; mod topk_stream; /// Hash aggregate modes +/// +/// See [`Accumulator::state`] for background information on multi-phase +/// aggregation and how these modes are used. #[derive(Debug, Copy, Clone, PartialEq, Eq)] pub enum AggregateMode { - /// Partial aggregate that can be applied in parallel across input partitions + /// Partial aggregate that can be applied in parallel across input + /// partitions. + /// + /// This is the first phase of a multi-phase aggregation. Partial, - /// Final aggregate that produces a single partition of output + /// Final aggregate that produces a single partition of output by combining + /// the output of multiple partial aggregates. + /// + /// This is the second phase of a multi-phase aggregation. Final, /// Final aggregate that works on pre-partitioned data. /// @@ -72,12 +81,15 @@ pub enum AggregateMode { /// Applies the entire logical aggregation operation in a single operator, /// as opposed to Partial / Final modes which apply the logical aggregation using /// two operators. + /// /// This mode requires that the input is a single partition (like Final) Single, /// Applies the entire logical aggregation operation in a single operator, /// as opposed to Partial / Final modes which apply the logical aggregation using /// two operators. - /// This mode requires that the input is partitioned by group key (like FinalPartitioned) + /// + /// This mode requires that the input is partitioned by group key (like + /// FinalPartitioned) SinglePartitioned, } @@ -2395,4 +2407,189 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_skip_aggregation_after_first_batch() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, true), + Field::new("val", DataType::Int32, true), + ])); + let df_schema = DFSchema::try_from(Arc::clone(&schema))?; + + let group_by = + PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); + + let aggr_expr: Vec> = + vec![create_aggregate_expr_with_dfschema( + &count_udaf(), + &[col("val", &schema)?], + &[datafusion_expr::col("val")], + &[], + &[], + &df_schema, + "COUNT(val)", + false, + false, + false, + )?]; + + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + ]; + + let input = Arc::new(MemoryExec::try_new( + &[input_data], + Arc::clone(&schema), + None, + )?); + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + schema, + )?); + + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + ScalarValue::Int64(Some(2)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + ScalarValue::Float64(Some(0.1)), + ); + + let ctx = TaskContext::default().with_session_config(session_config); + let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + + let expected = [ + "+-----+-------------------+", + "| key | COUNT(val)[count] |", + "+-----+-------------------+", + "| 1 | 1 |", + "| 2 | 1 |", + "| 3 | 1 |", + "| 2 | 1 |", + "| 3 | 1 |", + "| 4 | 1 |", + "+-----+-------------------+", + ]; + assert_batches_eq!(expected, &output); + + Ok(()) + } + + #[tokio::test] + async fn test_skip_aggregation_after_threshold() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, true), + Field::new("val", DataType::Int32, true), + ])); + let df_schema = DFSchema::try_from(Arc::clone(&schema))?; + + let group_by = + PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); + + let aggr_expr: Vec> = + vec![create_aggregate_expr_with_dfschema( + &count_udaf(), + &[col("val", &schema)?], + &[datafusion_expr::col("val")], + &[], + &[], + &df_schema, + "COUNT(val)", + false, + false, + false, + )?]; + + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + ]; + + let input = Arc::new(MemoryExec::try_new( + &[input_data], + Arc::clone(&schema), + None, + )?); + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + schema, + )?); + + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + ScalarValue::Int64(Some(5)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + ScalarValue::Float64(Some(0.1)), + ); + + let ctx = TaskContext::default().with_session_config(session_config); + let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + + let expected = [ + "+-----+-------------------+", + "| key | COUNT(val)[count] |", + "+-----+-------------------+", + "| 1 | 1 |", + "| 2 | 2 |", + "| 3 | 2 |", + "| 4 | 1 |", + "| 2 | 1 |", + "| 3 | 1 |", + "| 4 | 1 |", + "+-----+-------------------+", + ]; + assert_batches_eq!(expected, &output); + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 167ca7240750..62ed79dad4aa 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -39,7 +39,7 @@ use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; use arrow::datatypes::SchemaRef; use arrow_schema::SortOptions; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -62,6 +62,12 @@ pub(crate) enum ExecutionState { /// When producing output, the remaining rows to output are stored /// here and are sliced off as needed in batch_size chunks ProducingOutput(RecordBatch), + /// Produce intermediate aggregate state for each input row without + /// aggregation. + /// + /// See "partial aggregation" discussion on [`GroupedHashAggregateStream`] + SkippingAggregation, + /// All input has been consumed and all groups have been emitted Done, } @@ -90,6 +96,72 @@ struct SpillState { merging_group_by: PhysicalGroupBy, } +/// Tracks if the aggregate should skip partial aggregations +/// +/// See "partial aggregation" discussion on [`GroupedHashAggregateStream`] +struct SkipAggregationProbe { + /// Number of processed input rows + input_rows: usize, + /// Number of total group values for `input_rows` + num_groups: usize, + + /// Aggregation ratio check should be performed only when the + /// number of input rows exceeds this threshold + probe_rows_threshold: usize, + /// Maximum allowed value of `input_rows` / `num_groups` to + /// continue aggregation + probe_ratio_threshold: f64, + + /// Flag indicating that further data aggregation mey be skipped + should_skip: bool, + /// Flag indicating that further updates of `SkipAggregationProbe` + /// state won't make any effect + is_locked: bool, +} + +impl SkipAggregationProbe { + fn new(probe_rows_threshold: usize, probe_ratio_threshold: f64) -> Self { + Self { + input_rows: 0, + num_groups: 0, + probe_rows_threshold, + probe_ratio_threshold, + should_skip: false, + is_locked: false, + } + } + + /// Updates `SkipAggregationProbe` state: + /// - increments the number of input rows + /// - replaces the number of groups with the new value + /// - on `probe_rows_threshold` exceeded calculates + /// aggregation ratio and sets `should_skip` flag + /// - if `should_skip` is set, locks further state updates + fn update_state(&mut self, input_rows: usize, num_groups: usize) { + if self.is_locked { + return; + } + self.input_rows += input_rows; + self.num_groups = num_groups; + if self.input_rows >= self.probe_rows_threshold { + self.should_skip = self.num_groups as f64 / self.input_rows as f64 + >= self.probe_ratio_threshold; + self.is_locked = true; + } + } + + fn should_skip(&self) -> bool { + self.should_skip + } + + /// Provides an ability to externally set `should_skip` flag + /// to `false` and prohibit further state updates + fn forbid_skipping(&mut self) { + self.should_skip = false; + self.is_locked = true; + } +} + /// HashTable based Grouping Aggregator /// /// # Design Goals @@ -137,7 +209,7 @@ struct SpillState { /// of `x` and one accumulator for `SUM(y)`, specialized for the data /// type of `y`. /// -/// # Description +/// # Discussion /// /// [`group_values`] does not store any aggregate state inline. It only /// assigns "group indices", one for each (distinct) group value. The @@ -155,7 +227,25 @@ struct SpillState { /// /// [`group_values`]: Self::group_values /// -/// # Spilling +/// # Partial Aggregate and multi-phase grouping +/// +/// As described on [`Accumulator::state`], this operator is used in the context +/// "multi-phase" grouping when the mode is [`AggregateMode::Partial`]. +/// +/// An important optimization for multi-phase partial aggregation is to skip +/// partial aggregation when it is not effective enough to warrant the memory or +/// CPU cost, as is often the case for queries many distinct groups (high +/// cardinality group by). Memory is particularly important because each Partial +/// aggregator must store the intermediate state for each group. +/// +/// If the ratio of the number of groups to the number of input rows exceeds a +/// threshold, and [`GroupsAccumulator::supports_convert_to_state`] is +/// supported, this operator will stop applying Partial aggregation and directly +/// pass the input rows to the next aggregation phase. +/// +/// [`Accumulator::state`]: datafusion_expr::Accumulator::state +/// +/// # Spilling (to disk) /// /// The sizes of group values and accumulators can become large. Before that causes out of memory, /// this hash aggregator outputs partial states early for partial aggregation or spills to local @@ -275,6 +365,10 @@ pub(crate) struct GroupedHashAggregateStream { /// the `GroupedHashAggregateStream` operation immediately switches to /// output mode and emits all groups. group_values_soft_limit: Option, + + /// Optional probe for skipping data aggregation, if supported by + /// current stream. + skip_aggregation_probe: Option, } impl GroupedHashAggregateStream { @@ -365,6 +459,36 @@ impl GroupedHashAggregateStream { merging_group_by: PhysicalGroupBy::new_single(agg_group_by.expr.clone()), }; + // Skip aggregation is supported if: + // - aggregation mode is Partial + // - input is not ordered by GROUP BY expressions, + // since Final mode expects unique group values as its input + // - all accumulators support input batch to intermediate + // aggregate state conversion + // - there is only one GROUP BY expressions set + let skip_aggregation_probe = if agg.mode == AggregateMode::Partial + && matches!(group_ordering, GroupOrdering::None) + && accumulators + .iter() + .all(|acc| acc.supports_convert_to_state()) + && agg_group_by.is_single() + { + Some(SkipAggregationProbe::new( + context + .session_config() + .options() + .execution + .skip_partial_aggregation_probe_rows_threshold, + context + .session_config() + .options() + .execution + .skip_partial_aggregation_probe_ratio_threshold, + )) + } else { + None + }; + Ok(GroupedHashAggregateStream { schema: agg_schema, input, @@ -384,6 +508,7 @@ impl GroupedHashAggregateStream { runtime: context.runtime_env(), spill_state, group_values_soft_limit: agg.limit, + skip_aggregation_probe, }) } } @@ -434,12 +559,16 @@ impl Stream for GroupedHashAggregateStream { // new batch to aggregate Some(Ok(batch)) => { let timer = elapsed_compute.timer(); + let input_rows = batch.num_rows(); + // Make sure we have enough capacity for `batch`, otherwise spill extract_ok!(self.spill_previous_if_necessary(&batch)); // Do the grouping extract_ok!(self.group_aggregate_batch(batch)); + self.update_skip_aggregation_probe(input_rows); + // If we can begin emitting rows, do so, // otherwise keep consuming input assert!(!self.input_done); @@ -463,6 +592,8 @@ impl Stream for GroupedHashAggregateStream { extract_ok!(self.emit_early_if_necessary()); + extract_ok!(self.switch_to_skip_aggregation()); + timer.done(); } Some(Err(e)) => { @@ -476,6 +607,26 @@ impl Stream for GroupedHashAggregateStream { } } + ExecutionState::SkippingAggregation => { + match ready!(self.input.poll_next_unpin(cx)) { + Some(Ok(batch)) => { + let _timer = elapsed_compute.timer(); + let states = self.transform_to_states(batch)?; + return Poll::Ready(Some(Ok( + states.record_output(&self.baseline_metrics) + ))); + } + Some(Err(e)) => { + // inner had error, return to caller + return Poll::Ready(Some(Err(e))); + } + None => { + // inner is done, switching to `Done` state + self.exec_state = ExecutionState::Done; + } + } + } + ExecutionState::ProducingOutput(batch) => { // slice off a part of the batch, if needed let output_batch; @@ -484,6 +635,12 @@ impl Stream for GroupedHashAggregateStream { ( if self.input_done { ExecutionState::Done + } else if self + .skip_aggregation_probe + .as_ref() + .is_some_and(|probe| probe.should_skip()) + { + ExecutionState::SkippingAggregation } else { ExecutionState::ReadingInput }, @@ -797,4 +954,59 @@ impl GroupedHashAggregateStream { timer.done(); Ok(()) } + + // Updates skip aggregation probe state. + // In case stream has any spills, the probe is forcefully set to + // forbid aggregation skipping, and locked, since spilling resets + // total number of unique groups. + // + // Note: currently spilling is not supported for Partial aggregation + fn update_skip_aggregation_probe(&mut self, input_rows: usize) { + if let Some(probe) = self.skip_aggregation_probe.as_mut() { + if !self.spill_state.spills.is_empty() { + probe.forbid_skipping(); + } else { + probe.update_state(input_rows, self.group_values.len()); + } + }; + } + + // In case the probe indicates that aggregation may be + // skipped, forces stream to produce currently accumulated output. + fn switch_to_skip_aggregation(&mut self) -> Result<()> { + if let Some(probe) = self.skip_aggregation_probe.as_mut() { + if probe.should_skip() { + let batch = self.emit(EmitTo::All, false)?; + self.exec_state = ExecutionState::ProducingOutput(batch); + } + } + + Ok(()) + } + + // Transforms input batch to intermediate aggregate state, without grouping it + fn transform_to_states(&self, batch: RecordBatch) -> Result { + let group_values = evaluate_group_by(&self.group_by, &batch)?; + let input_values = evaluate_many(&self.aggregate_arguments, &batch)?; + let filter_values = evaluate_optional(&self.filter_expressions, &batch)?; + + let mut output = group_values.first().cloned().ok_or_else(|| { + internal_datafusion_err!("group_values expected to have at least one element") + })?; + + let iter = self + .accumulators + .iter() + .zip(input_values.iter()) + .zip(filter_values.iter()); + + for ((acc, values), opt_filter) in iter { + let opt_filter = opt_filter.as_ref().map(|filter| filter.as_boolean()); + output.extend(acc.convert_to_state(values, opt_filter)?); + } + + let states_batch = RecordBatch::try_new(self.schema(), output)?; + + Ok(states_batch) + } } diff --git a/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt b/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt new file mode 100644 index 000000000000..65efc24ec037 --- /dev/null +++ b/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt @@ -0,0 +1,324 @@ +# 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. + +# The main goal of these tests is to verify correctness of transforming +# input values to state by accumulators, supporting `convert_to_state`. + + +# 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'); + +# Prepare settings to skip partial aggregation from the beginning +statement ok +set datafusion.execution.skip_partial_aggregation_probe_rows_threshold = 0; + +statement ok +set datafusion.execution.skip_partial_aggregation_probe_ratio_threshold = 0.0; + +statement ok +set datafusion.execution.target_partitions = 2; + +statement ok +set datafusion.execution.batch_size = 1; + +statement ok +set datafusion.sql_parser.dialect = 'Postgres'; + +# Grouping by unique fields allows to check all accumulators +query ITIIII +SELECT c5, c1, + COUNT(), + COUNT(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END), + COUNT() FILTER (WHERE c1 = 'b'), + COUNT(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END) FILTER (WHERE c1 = 'b') +FROM aggregate_test_100 +GROUP BY 1, 2 ORDER BY 1 LIMIT 5; +---- +-2141999138 c 1 0 0 0 +-2141451704 a 1 1 0 0 +-2138770630 b 1 0 1 0 +-2117946883 d 1 0 0 0 +-2098805236 c 1 0 0 0 + +query ITIIII +SELECT c5, c1, + MIN(c5), + MIN(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END), + MIN(c5) FILTER (WHERE c1 = 'b'), + MIN(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END) FILTER (WHERE c1 = 'b') +FROM aggregate_test_100 +GROUP BY 1, 2 ORDER BY 1 LIMIT 5; +---- +-2141999138 c -2141999138 NULL NULL NULL +-2141451704 a -2141451704 -2141451704 NULL NULL +-2138770630 b -2138770630 NULL -2138770630 NULL +-2117946883 d -2117946883 NULL NULL NULL +-2098805236 c -2098805236 NULL NULL NULL + +query ITIIII +SELECT c5, c1, + MAX(c5), + MAX(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END), + MAX(c5) FILTER (WHERE c1 = 'b'), + MAX(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END) FILTER (WHERE c1 = 'b') +FROM aggregate_test_100 +GROUP BY 1, 2 ORDER BY 1 LIMIT 5; +---- +-2141999138 c -2141999138 NULL NULL NULL +-2141451704 a -2141451704 -2141451704 NULL NULL +-2138770630 b -2138770630 NULL -2138770630 NULL +-2117946883 d -2117946883 NULL NULL NULL +-2098805236 c -2098805236 NULL NULL NULL + +query ITIIII +SELECT c5, c1, + SUM(c5), + SUM(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END), + SUM(c5) FILTER (WHERE c1 = 'b'), + SUM(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END) FILTER (WHERE c1 = 'b') +FROM aggregate_test_100 +GROUP BY 1, 2 ORDER BY 1 LIMIT 5; +---- +-2141999138 c -2141999138 NULL NULL NULL +-2141451704 a -2141451704 -2141451704 NULL NULL +-2138770630 b -2138770630 NULL -2138770630 NULL +-2117946883 d -2117946883 NULL NULL NULL +-2098805236 c -2098805236 NULL NULL NULL + +# Prepare settings to always skip aggregation after couple of batches +statement ok +set datafusion.execution.skip_partial_aggregation_probe_rows_threshold = 10; + +statement ok +set datafusion.execution.skip_partial_aggregation_probe_ratio_threshold = 0.0; + +statement ok +set datafusion.execution.target_partitions = 2; + +statement ok +set datafusion.execution.batch_size = 4; + +# Inserting into nullable table with batch_size specified above +# to prevent creation on single in-memory batch +statement ok +CREATE TABLE aggregate_test_100_null ( + c2 TINYINT NOT NULL, + c5 INT NOT NULL, + c3 SMALLINT, + c11 FLOAT +); + +statement ok +INSERT INTO aggregate_test_100_null +SELECT + c2, + c5, + CASE WHEN c1 = 'e' THEN NULL ELSE c3 END as c3, + CASE WHEN c1 = 'a' THEN NULL ELSE c11 END as c11 +FROM aggregate_test_100; + +# Test count varchar / int / float +query IIII +SELECT c2, count(c1), count(c5), count(c11) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 22 22 22 +2 22 22 22 +3 19 19 19 +4 23 23 23 +5 14 14 14 + +# Test min / max for int / float +query IIIRR +SELECT c2, min(c5), max(c5), min(c11), max(c11) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 -1991133944 2143473091 0.064453244 0.89651865 +2 -2138770630 2053379412 0.055064857 0.8315913 +3 -2141999138 2030965207 0.034291923 0.9488028 +4 -1885422396 2064155045 0.028003037 0.7459874 +5 -2117946883 2025611582 0.12559289 0.87989986 + +# Test sum for int / float +query IIR +SELECT c2, sum(c5), sum(c11) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 -438598674 12.153253793716 +2 -8259865364 9.577824473381 +3 1956035476 9.590891361237 +4 16155718643 9.531112968922 +5 6449337880 7.074412226677 + +# Test count with nullable fields +query III +SELECT c2, count(c3), count(c11) FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 19 17 +2 17 19 +3 15 13 +4 16 19 +5 12 11 + +# Test min / max with nullable fields +query IIIRR +SELECT c2, min(c3), max(c3), min(c11), max(c11) FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 -99 125 0.064453244 0.89651865 +2 -117 122 0.09683716 0.8315913 +3 -101 123 0.034291923 0.94669616 +4 -117 123 0.028003037 0.7085086 +5 -101 118 0.12559289 0.87989986 + +# Test sum with nullable fields +query IIR +SELECT c2, sum(c3), sum(c11) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 367 12.153253793716 +2 184 9.577824473381 +3 395 9.590891361237 +4 29 9.531112968922 +5 -194 7.074412226677 + +# Enabling PG dialect for filtered aggregates tests +statement ok +set datafusion.sql_parser.dialect = 'Postgres'; + +# Test count with filter +query III +SELECT + c2, + count(c3) FILTER (WHERE c3 > 0), + count(c3) FILTER (WHERE c11 > 10) +FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 13 0 +2 13 0 +3 13 0 +4 13 0 +5 5 0 + +# Test min / max with filter +query III +SELECT + c2, + min(c3) FILTER (WHERE c3 > 0), + max(c3) FILTER (WHERE c3 < 0) +FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 12 -5 +2 1 -29 +3 13 -2 +4 3 -38 +5 36 -5 + +# Test sum with filter +query II +SELECT + c2, + sum(c3) FILTER (WHERE c1 != 'e' AND c3 > 0) +FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 612 +2 565 +3 466 +4 417 +5 284 + +# Test count with nullable fields and filter +query III +SELECT c2, + COUNT(c3) FILTER (WHERE c5 > 0), + COUNT(c11) FILTER(WHERE c5 > 0) +FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 11 6 +2 6 6 +3 8 6 +4 11 14 +5 8 7 + +# Test count with nullable fields and nullable filter +query III +SELECT c2, + COUNT(c3) FILTER (WHERE c11 > 0.5), + COUNT(c11) FILTER(WHERE c3 > 0) +FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 10 9 +2 7 8 +3 3 6 +4 3 7 +5 6 3 + +# Test min / max with nullable fields and filter +query IIIRR +SELECT c2, + MIN(c3) FILTER (WHERE c5 > 0), + MAX(c3) FILTER (WHERE c5 > 0), + MIN(c11) FILTER (WHERE c5 < 0), + MAX(c11) FILTER (WHERE c5 < 0) +FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 -99 103 0.2578469 0.89651865 +2 -48 93 0.09683716 0.8315913 +3 -76 123 0.034291923 0.94669616 +4 -117 123 0.06563997 0.57360977 +5 -94 68 0.12559289 0.75173044 + +# Test min / max with nullable fields and nullable filter +query III +SELECT c2, + MIN(c3) FILTER (WHERE c11 > 0.5), + MAX(c3) FILTER (WHERE c11 > 0.5) +FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 -99 125 +2 -106 122 +3 -76 73 +4 -117 47 +5 -82 118 + +# Test sum with nullable field and nullable / non-nullable filters +query IIIRR +SELECT c2, + SUM(c3) FILTER (WHERE c5 > 0), + SUM(c3) FILTER (WHERE c11 < 0.5), + SUM(c11) FILTER (WHERE c5 < 0), + SUM(c11) FILTER (WHERE c3 > 0) +FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; +---- +1 -3 77 7.214695632458 5.085060358047 +2 100 77 6.197732746601 3.150197088718 +3 109 211 2.80575042963 2.80632930994 +4 -171 56 2.10740506649 1.939846396446 +5 -86 -76 1.8741710186 1.600569307804 diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index fef7bfe82174..0cbbbf3c608c 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -207,6 +207,8 @@ datafusion.execution.parquet.statistics_enabled page datafusion.execution.parquet.write_batch_size 1024 datafusion.execution.parquet.writer_version 1.0 datafusion.execution.planning_concurrency 13 +datafusion.execution.skip_partial_aggregation_probe_ratio_threshold 0.8 +datafusion.execution.skip_partial_aggregation_probe_rows_threshold 100000 datafusion.execution.soft_max_rows_per_output_file 50000000 datafusion.execution.sort_in_place_threshold_bytes 1048576 datafusion.execution.sort_spill_reservation_bytes 10485760 @@ -294,6 +296,8 @@ datafusion.execution.parquet.statistics_enabled page (writing) Sets if statistic datafusion.execution.parquet.write_batch_size 1024 (writing) Sets write_batch_size in bytes datafusion.execution.parquet.writer_version 1.0 (writing) Sets parquet writer version valid values are "1.0" and "2.0" datafusion.execution.planning_concurrency 13 Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system +datafusion.execution.skip_partial_aggregation_probe_ratio_threshold 0.8 Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input +datafusion.execution.skip_partial_aggregation_probe_rows_threshold 100000 Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode datafusion.execution.soft_max_rows_per_output_file 50000000 Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max datafusion.execution.sort_in_place_threshold_bytes 1048576 When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. datafusion.execution.sort_spill_reservation_bytes 10485760 Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 78d0d7b0239f..badd07822ac2 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -89,6 +89,8 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | | datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | | datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From 682bc2eff2a428482a83fba0f304facb458e4428 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 5 Aug 2024 19:00:31 +0800 Subject: [PATCH 03/34] Improve log func stability (#11808) --- datafusion/functions/src/math/log.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index 0e181aa61250..ad7cff1f7149 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -261,10 +261,10 @@ mod tests { .expect("failed to convert result to a Float64Array"); assert_eq!(floats.len(), 4); - assert_eq!(floats.value(0), 3.0); - assert_eq!(floats.value(1), 2.0); - assert_eq!(floats.value(2), 4.0); - assert_eq!(floats.value(3), 4.0); + assert!((floats.value(0) - 3.0).abs() < 1e-10); + assert!((floats.value(1) - 2.0).abs() < 1e-10); + assert!((floats.value(2) - 4.0).abs() < 1e-10); + assert!((floats.value(3) - 4.0).abs() < 1e-10); } ColumnarValue::Scalar(_) => { panic!("Expected an array value") @@ -291,10 +291,10 @@ mod tests { .expect("failed to convert result to a Float32Array"); assert_eq!(floats.len(), 4); - assert_eq!(floats.value(0), 3.0); - assert_eq!(floats.value(1), 2.0); - assert_eq!(floats.value(2), 4.0); - assert_eq!(floats.value(3), 4.0); + assert!((floats.value(0) - 3.0).abs() < f32::EPSILON); + assert!((floats.value(1) - 2.0).abs() < f32::EPSILON); + assert!((floats.value(2) - 4.0).abs() < f32::EPSILON); + assert!((floats.value(3) - 4.0).abs() < f32::EPSILON); } ColumnarValue::Scalar(_) => { panic!("Expected an array value") From 6aad19fad370cccf4caa16fcca552bf246f59d93 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 5 Aug 2024 15:59:12 +0300 Subject: [PATCH 04/34] add valid distinct case for aggregate.slt (#11814) --- .../src/replace_distinct_aggregate.rs | 9 +++--- .../sqllogictest/test_files/aggregate.slt | 28 +++++++++++++++++++ 2 files changed, 33 insertions(+), 4 deletions(-) diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index f73eeacfbf0e..c887192f6370 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -82,10 +82,11 @@ impl OptimizerRule for ReplaceDistinctWithAggregate { for dep in input.schema().functional_dependencies().iter() { // If distinct is exactly the same with a previous GROUP BY, we can // simply remove it: - if dep.source_indices[..field_count] - .iter() - .enumerate() - .all(|(idx, f_idx)| idx == *f_idx) + if dep.source_indices.len() >= field_count + && dep.source_indices[..field_count] + .iter() + .enumerate() + .all(|(idx, f_idx)| idx == *f_idx) { return Ok(Transformed::yes(input.as_ref().clone())); } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index abeeb767b948..6513258f879e 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4521,6 +4521,34 @@ false true NULL +# +# Add valid distinct case as aggregation plan test +# + +query TT +EXPLAIN SELECT DISTINCT c3, min(c1) FROM aggregate_test_100 group by c3 limit 5; +---- +logical_plan +01)Limit: skip=0, fetch=5 +02)--Aggregate: groupBy=[[aggregate_test_100.c3, MIN(aggregate_test_100.c1)]], aggr=[[]] +03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[MIN(aggregate_test_100.c1)]] +04)------TableScan: aggregate_test_100 projection=[c1, c3] +physical_plan +01)GlobalLimitExec: skip=0, fetch=5 +02)--CoalescePartitionsExec +03)----LocalLimitExec: fetch=5 +04)------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, MIN(aggregate_test_100.c1)@1 as MIN(aggregate_test_100.c1)], aggr=[], lim=[5] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([c3@0, MIN(aggregate_test_100.c1)@1], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c3@0 as c3, MIN(aggregate_test_100.c1)@1 as MIN(aggregate_test_100.c1)], aggr=[], lim=[5] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[MIN(aggregate_test_100.c1)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 +11)--------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[MIN(aggregate_test_100.c1)] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true + + # # Push limit into distinct group-by aggregation tests # From 45d85b1d18ce33e2b42e8d3a5087addd8399a116 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Mon, 5 Aug 2024 10:49:46 -0400 Subject: [PATCH 05/34] Change name of MAX/MIN udaf to lowercase max/min (#11795) * Only row 8 diff * MAX -> max, MIN -> min * Updating tests * Removed aliases --- datafusion/core/src/dataframe/mod.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 6 +- datafusion/core/src/lib.rs | 4 +- .../core/tests/custom_sources_cases/mod.rs | 4 +- .../core/tests/expr_api/parse_sql_expr.rs | 6 +- datafusion/core/tests/sql/explain_analyze.rs | 6 +- datafusion/expr/src/expr_rewriter/order_by.rs | 12 +- datafusion/expr/src/test/function_stub.rs | 12 +- datafusion/functions-aggregate/src/min_max.rs | 12 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/optimize_projections/mod.rs | 32 +- datafusion/optimizer/src/push_down_limit.rs | 6 +- .../optimizer/src/scalar_subquery_to_join.rs | 156 ++++---- .../simplify_expressions/simplify_exprs.rs | 2 +- .../src/single_distinct_to_groupby.rs | 22 +- datafusion/physical-expr/src/aggregate/mod.rs | 1 - .../src/aggregate/moving_min_max.rs | 335 ------------------ datafusion/sql/tests/cases/plan_to_sql.rs | 4 +- datafusion/sql/tests/sql_integration.rs | 312 ++++++++-------- .../sqllogictest/test_files/aggregate.slt | 18 +- .../test_files/aggregates_topk.slt | 66 ++-- datafusion/sqllogictest/test_files/expr.slt | 8 +- .../sqllogictest/test_files/group_by.slt | 44 +-- .../sqllogictest/test_files/tpch/q15.slt.part | 12 +- .../sqllogictest/test_files/tpch/q2.slt.part | 16 +- datafusion/sqllogictest/test_files/union.slt | 8 +- datafusion/sqllogictest/test_files/update.slt | 4 +- datafusion/sqllogictest/test_files/window.slt | 118 +++--- .../tests/cases/consumer_integration.rs | 2 +- .../library-user-guide/using-the-sql-api.md | 4 +- 30 files changed, 446 insertions(+), 790 deletions(-) delete mode 100644 datafusion/physical-expr/src/aggregate/moving_min_max.rs diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 6ec44b33f89e..c4c5a4aa0834 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2060,7 +2060,7 @@ mod tests { assert_batches_sorted_eq!( ["+----+-----------------------------+-----------------------------+-----------------------------+-----------------------------+-------------------------------+----------------------------------------+", - "| c1 | MIN(aggregate_test_100.c12) | MAX(aggregate_test_100.c12) | avg(aggregate_test_100.c12) | sum(aggregate_test_100.c12) | count(aggregate_test_100.c12) | count(DISTINCT aggregate_test_100.c12) |", + "| c1 | min(aggregate_test_100.c12) | max(aggregate_test_100.c12) | avg(aggregate_test_100.c12) | sum(aggregate_test_100.c12) | count(aggregate_test_100.c12) | count(DISTINCT aggregate_test_100.c12) |", "+----+-----------------------------+-----------------------------+-----------------------------+-----------------------------+-------------------------------+----------------------------------------+", "| a | 0.02182578039211991 | 0.9800193410444061 | 0.48754517466109415 | 10.238448667882977 | 21 | 21 |", "| b | 0.04893135681998029 | 0.9185813970744787 | 0.41040709263815384 | 7.797734760124923 | 19 | 19 |", diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 24704bc794c2..e6bb1483e256 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -159,7 +159,7 @@ where /// assert_batches_eq!( /// &[ /// "+---+----------------+", -/// "| a | MIN(?table?.b) |", +/// "| a | min(?table?.b) |", /// "+---+----------------+", /// "| 1 | 2 |", /// "+---+----------------+", @@ -182,14 +182,14 @@ where /// let mut ctx = SessionContext::new(); /// ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?; /// let results = ctx -/// .sql("SELECT a, MIN(b) FROM example GROUP BY a LIMIT 100") +/// .sql("SELECT a, min(b) FROM example GROUP BY a LIMIT 100") /// .await? /// .collect() /// .await?; /// assert_batches_eq!( /// &[ /// "+---+----------------+", -/// "| a | MIN(example.b) |", +/// "| a | min(example.b) |", /// "+---+----------------+", /// "| 1 | 2 |", /// "+---+----------------+", diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 3bb0636652c0..d4b82f288bdd 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -76,7 +76,7 @@ //! //! let expected = vec![ //! "+---+----------------+", -//! "| a | MIN(?table?.b) |", +//! "| a | min(?table?.b) |", //! "+---+----------------+", //! "| 1 | 2 |", //! "+---+----------------+" @@ -114,7 +114,7 @@ //! //! let expected = vec![ //! "+---+----------------+", -//! "| a | MIN(example.b) |", +//! "| a | min(example.b) |", //! "+---+----------------+", //! "| 1 | 2 |", //! "+---+----------------+" diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index 673fafe55b53..c12dd4e1b0ee 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -284,8 +284,8 @@ async fn optimizers_catch_all_statistics() { let expected = RecordBatch::try_new( Arc::new(Schema::new(vec![ Field::new("count(*)", DataType::Int64, false), - Field::new("MIN(test.c1)", DataType::Int32, false), - Field::new("MAX(test.c1)", DataType::Int32, false), + Field::new("min(test.c1)", DataType::Int32, false), + Field::new("max(test.c1)", DataType::Int32, false), ])), vec![ Arc::new(Int64Array::from(vec![4])), diff --git a/datafusion/core/tests/expr_api/parse_sql_expr.rs b/datafusion/core/tests/expr_api/parse_sql_expr.rs index a3defceee247..cc049f0004d9 100644 --- a/datafusion/core/tests/expr_api/parse_sql_expr.rs +++ b/datafusion/core/tests/expr_api/parse_sql_expr.rs @@ -49,9 +49,9 @@ async fn round_trip_parse_sql_expr() -> Result<()> { "((a = 10) AND b NOT IN (20, 30))", "sum(a)", "(sum(a) + 1)", - "(MIN(a) + MAX(b))", - "(MIN(a) + (MAX(b) * sum(c)))", - "(MIN(a) + ((MAX(b) * sum(c)) / 10))", + "(min(a) + max(b))", + "(min(a) + (max(b) * sum(c)))", + "(min(a) + ((max(b) * sum(c)) / 10))", ]; for test in tests { diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 07be00fc3515..4c1f5efaf989 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -615,11 +615,11 @@ async fn test_physical_plan_display_indent() { "GlobalLimitExec: skip=0, fetch=10", " SortPreservingMergeExec: [the_min@2 DESC], fetch=10", " SortExec: TopK(fetch=10), expr=[the_min@2 DESC], preserve_partitioning=[true]", - " ProjectionExec: expr=[c1@0 as c1, MAX(aggregate_test_100.c12)@1 as MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)@2 as the_min]", - " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)]", + " ProjectionExec: expr=[c1@0 as c1, max(aggregate_test_100.c12)@1 as max(aggregate_test_100.c12), min(aggregate_test_100.c12)@2 as the_min]", + " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", - " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)]", + " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: c12@1 < 10", " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index 2efdcae1a790..3d79caa21fde 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -25,7 +25,7 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Column, Result}; /// Rewrite sort on aggregate expressions to sort on the column of aggregate output -/// For example, `max(x)` is written to `col("MAX(x)")` +/// For example, `max(x)` is written to `col("max(x)")` pub fn rewrite_sort_cols_by_aggs( exprs: impl IntoIterator>, plan: &LogicalPlan, @@ -108,7 +108,7 @@ fn rewrite_in_terms_of_projection( }; // expr is an actual expr like min(t.c2), but we are looking - // for a column with the same "MIN(C2)", so translate there + // for a column with the same "min(C2)", so translate there let name = normalized_expr.display_name()?; let search_col = Expr::Column(Column { @@ -237,15 +237,15 @@ mod test { expected: sort(col("c1")), }, TestCase { - desc: r#"min(c2) --> "MIN(c2)" -- (column *named* "min(t.c2)"!)"#, + desc: r#"min(c2) --> "min(c2)" -- (column *named* "min(t.c2)"!)"#, input: sort(min(col("c2"))), - expected: sort(col("MIN(t.c2)")), + expected: sort(col("min(t.c2)")), }, TestCase { - desc: r#"c1 + min(c2) --> "c1 + MIN(c2)" -- (column *named* "min(t.c2)"!)"#, + desc: r#"c1 + min(c2) --> "c1 + min(c2)" -- (column *named* "min(t.c2)"!)"#, input: sort(col("c1") + min(col("c2"))), // should be "c1" not t.c1 - expected: sort(col("c1") + col("MIN(t.c2)")), + expected: sort(col("c1") + col("min(t.c2)")), }, TestCase { desc: r#"avg(c3) --> "avg(t.c3)" as average (column *named* "avg(t.c3)", aliased)"#, diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index 72b73ccee44f..22af84db3f86 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -305,7 +305,6 @@ pub fn min(expr: Expr) -> Expr { /// Testing stub implementation of Min aggregate pub struct Min { signature: Signature, - aliases: Vec, } impl std::fmt::Debug for Min { @@ -326,7 +325,6 @@ impl Default for Min { impl Min { pub fn new() -> Self { Self { - aliases: vec!["min".to_string()], signature: Signature::variadic_any(Volatility::Immutable), } } @@ -338,7 +336,7 @@ impl AggregateUDFImpl for Min { } fn name(&self) -> &str { - "MIN" + "min" } fn signature(&self) -> &Signature { @@ -358,7 +356,7 @@ impl AggregateUDFImpl for Min { } fn aliases(&self) -> &[String] { - &self.aliases + &[] } fn create_groups_accumulator( @@ -392,7 +390,6 @@ pub fn max(expr: Expr) -> Expr { /// Testing stub implementation of MAX aggregate pub struct Max { signature: Signature, - aliases: Vec, } impl std::fmt::Debug for Max { @@ -413,7 +410,6 @@ impl Default for Max { impl Max { pub fn new() -> Self { Self { - aliases: vec!["max".to_string()], signature: Signature::variadic_any(Volatility::Immutable), } } @@ -425,7 +421,7 @@ impl AggregateUDFImpl for Max { } fn name(&self) -> &str { - "MAX" + "max" } fn signature(&self) -> &Signature { @@ -445,7 +441,7 @@ impl AggregateUDFImpl for Max { } fn aliases(&self) -> &[String] { - &self.aliases + &[] } fn create_groups_accumulator( diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 4d743983411d..18028e358b21 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -85,14 +85,12 @@ fn get_min_max_result_type(input_types: &[DataType]) -> Result> { // MAX aggregate UDF #[derive(Debug)] pub struct Max { - aliases: Vec, signature: Signature, } impl Max { pub fn new() -> Self { Self { - aliases: vec!["max".to_owned()], signature: Signature::user_defined(Volatility::Immutable), } } @@ -146,7 +144,7 @@ impl AggregateUDFImpl for Max { } fn name(&self) -> &str { - "MAX" + "max" } fn signature(&self) -> &Signature { @@ -162,7 +160,7 @@ impl AggregateUDFImpl for Max { } fn aliases(&self) -> &[String] { - &self.aliases + &[] } fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { @@ -891,14 +889,12 @@ impl Accumulator for SlidingMaxAccumulator { #[derive(Debug)] pub struct Min { signature: Signature, - aliases: Vec, } impl Min { pub fn new() -> Self { Self { signature: Signature::user_defined(Volatility::Immutable), - aliases: vec!["min".to_owned()], } } } @@ -915,7 +911,7 @@ impl AggregateUDFImpl for Min { } fn name(&self) -> &str { - "MIN" + "min" } fn signature(&self) -> &Signature { @@ -931,7 +927,7 @@ impl AggregateUDFImpl for Min { } fn aliases(&self) -> &[String] { - &self.aliases + &[] } fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index e2da6c66abc4..8ff00917dcb1 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -271,7 +271,7 @@ mod tests { .build()?; let expected = "Projection: count(Int64(1)) AS count(*) [count(*):Int64]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(count(Int64(1))) AS MAX(count(*))]] [MAX(count(*)):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(count(Int64(1))) AS max(count(*))]] [max(count(*)):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_plan_eq(plan, expected) } diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index a307d0ae0a0b..d014b9149aab 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -1361,7 +1361,7 @@ mod tests { .aggregate(Vec::::new(), vec![max(col("b"))])? .build()?; - let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ + let expected = "Aggregate: groupBy=[[]], aggr=[[max(test.b)]]\ \n TableScan: test projection=[b]"; assert_optimized_plan_equal(plan, expected) @@ -1375,7 +1375,7 @@ mod tests { .aggregate(vec![col("c")], vec![max(col("b"))])? .build()?; - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.b)]]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[max(test.b)]]\ \n TableScan: test projection=[b, c]"; assert_optimized_plan_equal(plan, expected) @@ -1390,7 +1390,7 @@ mod tests { .aggregate(vec![col("c")], vec![max(col("b"))])? .build()?; - let expected = "Aggregate: groupBy=[[a.c]], aggr=[[MAX(a.b)]]\ + let expected = "Aggregate: groupBy=[[a.c]], aggr=[[max(a.b)]]\ \n SubqueryAlias: a\ \n TableScan: test projection=[b, c]"; @@ -1406,7 +1406,7 @@ mod tests { .aggregate(Vec::::new(), vec![max(col("b"))])? .build()?; - let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ + let expected = "Aggregate: groupBy=[[]], aggr=[[max(test.b)]]\ \n Projection: test.b\ \n Filter: test.c > Int32(1)\ \n TableScan: test projection=[b, c]"; @@ -1422,7 +1422,7 @@ mod tests { // "tag.one", not a column named "one" in a table named "tag"): // // Projection: tag.one - // Aggregate: groupBy=[], aggr=[MAX("tag.one") AS "tag.one"] + // Aggregate: groupBy=[], aggr=[max("tag.one") AS "tag.one"] // TableScan let plan = table_scan(Some("m4"), &schema, None)? .aggregate( @@ -1433,7 +1433,7 @@ mod tests { .build()?; let expected = "\ - Aggregate: groupBy=[[]], aggr=[[MAX(m4.tag.one) AS tag.one]]\ + Aggregate: groupBy=[[]], aggr=[[max(m4.tag.one) AS tag.one]]\ \n TableScan: m4 projection=[tag.one]"; assert_optimized_plan_equal(plan, expected) @@ -1768,11 +1768,11 @@ mod tests { .aggregate(vec![col("c")], vec![max(col("a"))])? .build()?; - assert_fields_eq(&plan, vec!["c", "MAX(test.a)"]); + assert_fields_eq(&plan, vec!["c", "max(test.a)"]); let plan = optimize(plan).expect("failed to optimize plan"); let expected = "\ - Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.a)]]\ + Aggregate: groupBy=[[test.c]], aggr=[[max(test.a)]]\ \n Filter: test.c > Int32(1)\ \n Projection: test.c, test.a\ \n TableScan: test projection=[a, c]"; @@ -1862,14 +1862,14 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a"), col("c")], vec![max(col("b")), min(col("b"))])? .filter(col("c").gt(lit(1)))? - .project(vec![col("c"), col("a"), col("MAX(test.b)")])? + .project(vec![col("c"), col("a"), col("max(test.b)")])? .build()?; - assert_fields_eq(&plan, vec!["c", "a", "MAX(test.b)"]); + assert_fields_eq(&plan, vec!["c", "a", "max(test.b)"]); - let expected = "Projection: test.c, test.a, MAX(test.b)\ + let expected = "Projection: test.c, test.a, max(test.b)\ \n Filter: test.c > Int32(1)\ - \n Aggregate: groupBy=[[test.a, test.c]], aggr=[[MAX(test.b)]]\ + \n Aggregate: groupBy=[[test.a, test.c]], aggr=[[max(test.b)]]\ \n TableScan: test projection=[a, b, c]"; assert_optimized_plan_equal(plan, expected) @@ -1937,10 +1937,10 @@ mod tests { .project(vec![col1, col2])? .build()?; - let expected = "Projection: MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MAX(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n Projection: test.b, MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + let expected = "Projection: max(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, max(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[max(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + \n Projection: test.b, max(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[max(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: test projection=[a, b]"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index d7da3871ee89..612aac1d152d 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -375,7 +375,7 @@ mod test { // Limit should *not* push down aggregate node let expected = "Limit: skip=0, fetch=1000\ - \n Aggregate: groupBy=[[test.a]], aggr=[[MAX(test.b)]]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[max(test.b)]]\ \n TableScan: test"; assert_optimized_plan_equal(plan, expected) @@ -447,7 +447,7 @@ mod test { // Limit should use deeper LIMIT 1000, but Limit 10 shouldn't push down aggregation let expected = "Limit: skip=0, fetch=10\ - \n Aggregate: groupBy=[[test.a]], aggr=[[MAX(test.b)]]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[max(test.b)]]\ \n Limit: skip=0, fetch=1000\ \n TableScan: test, fetch=1000"; @@ -548,7 +548,7 @@ mod test { // Limit should *not* push down aggregate node let expected = "Limit: skip=10, fetch=1000\ - \n Aggregate: groupBy=[[test.a]], aggr=[[MAX(test.b)]]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[max(test.b)]]\ \n TableScan: test"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index fbec675f6fc4..3c66da21aff6 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -422,17 +422,17 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: Int32(1) < __scalar_sq_1.MAX(orders.o_custkey) AND Int32(1) < __scalar_sq_2.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: __scalar_sq_2.o_custkey = customer.c_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: __scalar_sq_1.o_custkey = customer.c_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Filter: Int32(1) < __scalar_sq_1.max(orders.o_custkey) AND Int32(1) < __scalar_sq_2.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: __scalar_sq_2.o_custkey = customer.c_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: __scalar_sq_1.o_custkey = customer.c_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ - \n SubqueryAlias: __scalar_sq_2 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_2 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], @@ -519,12 +519,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n Filter: orders.o_orderkey = Int32(1) [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; @@ -557,12 +557,12 @@ mod tests { // it will optimize, but fail for the same reason the unoptimized query would let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ - \n Left Join: [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ + \n Left Join: [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N]\ - \n Projection: MAX(orders.o_custkey) [MAX(orders.o_custkey):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(orders.o_custkey)]] [MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], @@ -589,12 +589,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ - \n Left Join: [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ + \n Left Join: [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N]\ - \n Projection: MAX(orders.o_custkey) [MAX(orders.o_custkey):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(orders.o_custkey)]] [MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ \n Filter: orders.o_custkey = orders.o_custkey [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; @@ -719,7 +719,7 @@ mod tests { .eq(col("orders.o_custkey")), )? .aggregate(Vec::::new(), vec![max(col("orders.o_custkey"))])? - .project(vec![col("MAX(orders.o_custkey)").add(lit(1))])? + .project(vec![col("max(orders.o_custkey)").add(lit(1))])? .build()?, ); @@ -729,12 +729,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) + Int32(1) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) + Int32(1) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey) + Int32(1), orders.o_custkey [MAX(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey) + Int32(1), orders.o_custkey [max(orders.o_custkey) + Int32(1):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -795,12 +795,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey >= __scalar_sq_1.MAX(orders.o_custkey) AND customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Filter: customer.c_custkey >= __scalar_sq_1.max(orders.o_custkey) AND customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -834,12 +834,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) AND customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) AND customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -874,12 +874,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) OR customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) OR customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -907,12 +907,12 @@ mod tests { .build()?; let expected = "Projection: test.c [c:UInt32]\ - \n Filter: test.c < __scalar_sq_1.MIN(sq.c) [a:UInt32, b:UInt32, c:UInt32, MIN(sq.c):UInt32;N, a:UInt32;N]\ - \n Left Join: Filter: test.a = __scalar_sq_1.a [a:UInt32, b:UInt32, c:UInt32, MIN(sq.c):UInt32;N, a:UInt32;N]\ + \n Filter: test.c < __scalar_sq_1.min(sq.c) [a:UInt32, b:UInt32, c:UInt32, min(sq.c):UInt32;N, a:UInt32;N]\ + \n Left Join: Filter: test.a = __scalar_sq_1.a [a:UInt32, b:UInt32, c:UInt32, min(sq.c):UInt32;N, a:UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ - \n SubqueryAlias: __scalar_sq_1 [MIN(sq.c):UInt32;N, a:UInt32]\ - \n Projection: MIN(sq.c), sq.a [MIN(sq.c):UInt32;N, a:UInt32]\ - \n Aggregate: groupBy=[[sq.a]], aggr=[[MIN(sq.c)]] [a:UInt32, MIN(sq.c):UInt32;N]\ + \n SubqueryAlias: __scalar_sq_1 [min(sq.c):UInt32;N, a:UInt32]\ + \n Projection: min(sq.c), sq.a [min(sq.c):UInt32;N, a:UInt32]\ + \n Aggregate: groupBy=[[sq.a]], aggr=[[min(sq.c)]] [a:UInt32, min(sq.c):UInt32;N]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -939,12 +939,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey < __scalar_sq_1.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ - \n Left Join: [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ + \n Filter: customer.c_custkey < __scalar_sq_1.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ + \n Left Join: [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N]\ - \n Projection: MAX(orders.o_custkey) [MAX(orders.o_custkey):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(orders.o_custkey)]] [MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -970,12 +970,12 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey = __scalar_sq_1.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ - \n Left Join: [c_custkey:Int64, c_name:Utf8, MAX(orders.o_custkey):Int64;N]\ + \n Filter: customer.c_custkey = __scalar_sq_1.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ + \n Left Join: [c_custkey:Int64, c_name:Utf8, max(orders.o_custkey):Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MAX(orders.o_custkey):Int64;N]\ - \n Projection: MAX(orders.o_custkey) [MAX(orders.o_custkey):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(orders.o_custkey)]] [MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -1022,17 +1022,17 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey BETWEEN __scalar_sq_1.MIN(orders.o_custkey) AND __scalar_sq_2.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MIN(orders.o_custkey):Int64;N, o_custkey:Int64;N, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_2.o_custkey [c_custkey:Int64, c_name:Utf8, MIN(orders.o_custkey):Int64;N, o_custkey:Int64;N, MAX(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ - \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, MIN(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Filter: customer.c_custkey BETWEEN __scalar_sq_1.min(orders.o_custkey) AND __scalar_sq_2.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, min(orders.o_custkey):Int64;N, o_custkey:Int64;N, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_2.o_custkey [c_custkey:Int64, c_name:Utf8, min(orders.o_custkey):Int64;N, o_custkey:Int64;N, max(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ + \n Left Join: Filter: customer.c_custkey = __scalar_sq_1.o_custkey [c_custkey:Int64, c_name:Utf8, min(orders.o_custkey):Int64;N, o_custkey:Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MIN(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MIN(orders.o_custkey), orders.o_custkey [MIN(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MIN(orders.o_custkey)]] [o_custkey:Int64, MIN(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [min(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: min(orders.o_custkey), orders.o_custkey [min(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[min(orders.o_custkey)]] [o_custkey:Int64, min(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ - \n SubqueryAlias: __scalar_sq_2 [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Projection: MAX(orders.o_custkey), orders.o_custkey [MAX(orders.o_custkey):Int64;N, o_custkey:Int64]\ - \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[MAX(orders.o_custkey)]] [o_custkey:Int64, MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_2 [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Projection: max(orders.o_custkey), orders.o_custkey [max(orders.o_custkey):Int64;N, o_custkey:Int64]\ + \n Aggregate: groupBy=[[orders.o_custkey]], aggr=[[max(orders.o_custkey)]] [o_custkey:Int64, max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( @@ -1071,17 +1071,17 @@ mod tests { .build()?; let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ - \n Filter: customer.c_custkey BETWEEN __scalar_sq_1.MIN(orders.o_custkey) AND __scalar_sq_2.MAX(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, MIN(orders.o_custkey):Int64;N, MAX(orders.o_custkey):Int64;N]\ - \n Left Join: [c_custkey:Int64, c_name:Utf8, MIN(orders.o_custkey):Int64;N, MAX(orders.o_custkey):Int64;N]\ - \n Left Join: [c_custkey:Int64, c_name:Utf8, MIN(orders.o_custkey):Int64;N]\ + \n Filter: customer.c_custkey BETWEEN __scalar_sq_1.min(orders.o_custkey) AND __scalar_sq_2.max(orders.o_custkey) [c_custkey:Int64, c_name:Utf8, min(orders.o_custkey):Int64;N, max(orders.o_custkey):Int64;N]\ + \n Left Join: [c_custkey:Int64, c_name:Utf8, min(orders.o_custkey):Int64;N, max(orders.o_custkey):Int64;N]\ + \n Left Join: [c_custkey:Int64, c_name:Utf8, min(orders.o_custkey):Int64;N]\ \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ - \n SubqueryAlias: __scalar_sq_1 [MIN(orders.o_custkey):Int64;N]\ - \n Projection: MIN(orders.o_custkey) [MIN(orders.o_custkey):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MIN(orders.o_custkey)]] [MIN(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_1 [min(orders.o_custkey):Int64;N]\ + \n Projection: min(orders.o_custkey) [min(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[min(orders.o_custkey)]] [min(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ - \n SubqueryAlias: __scalar_sq_2 [MAX(orders.o_custkey):Int64;N]\ - \n Projection: MAX(orders.o_custkey) [MAX(orders.o_custkey):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(orders.o_custkey)]] [MAX(orders.o_custkey):Int64;N]\ + \n SubqueryAlias: __scalar_sq_2 [max(orders.o_custkey):Int64;N]\ + \n Projection: max(orders.o_custkey) [max(orders.o_custkey):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[max(orders.o_custkey)]] [max(orders.o_custkey):Int64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index cb9ec3024d93..e13ce382a3e0 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -401,7 +401,7 @@ mod tests { .build()?; let expected = "\ - Aggregate: groupBy=[[test.a, test.c]], aggr=[[MAX(test.b) AS MAX(test.b = Boolean(true)), MIN(test.b)]]\ + Aggregate: groupBy=[[test.a, test.c]], aggr=[[max(test.b) AS max(test.b = Boolean(true)), min(test.b)]]\ \n Projection: test.a, test.c, test.b\ \n TableScan: test"; diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 9a0fab14d3e0..5b4395792447 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -324,7 +324,7 @@ mod tests { // Do nothing let expected = - "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]] [MAX(test.b):UInt32;N]\ + "Aggregate: groupBy=[[]], aggr=[[max(test.b)]] [max(test.b):UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -467,8 +467,8 @@ mod tests { )? .build()?; // Should work - let expected = "Projection: test.a, count(alias1) AS count(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, count(DISTINCT test.b):Int64, MAX(DISTINCT test.b):UInt32;N]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[count(alias1), MAX(alias1)]] [a:UInt32, count(alias1):Int64, MAX(alias1):UInt32;N]\ + let expected = "Projection: test.a, count(alias1) AS count(DISTINCT test.b), max(alias1) AS max(DISTINCT test.b) [a:UInt32, count(DISTINCT test.b):Int64, max(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[count(alias1), max(alias1)]] [a:UInt32, count(alias1):Int64, max(alias1):UInt32;N]\ \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; @@ -525,8 +525,8 @@ mod tests { )? .build()?; // Should work - let expected = "Projection: test.a, sum(alias2) AS sum(test.c), count(alias1) AS count(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, sum(test.c):UInt64;N, count(DISTINCT test.b):Int64, MAX(DISTINCT test.b):UInt32;N]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[sum(alias2), count(alias1), MAX(alias1)]] [a:UInt32, sum(alias2):UInt64;N, count(alias1):Int64, MAX(alias1):UInt32;N]\ + let expected = "Projection: test.a, sum(alias2) AS sum(test.c), count(alias1) AS count(DISTINCT test.b), max(alias1) AS max(DISTINCT test.b) [a:UInt32, sum(test.c):UInt64;N, count(DISTINCT test.b):Int64, max(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[sum(alias2), count(alias1), max(alias1)]] [a:UInt32, sum(alias2):UInt64;N, count(alias1):Int64, max(alias1):UInt32;N]\ \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[sum(test.c) AS alias2]] [a:UInt32, alias1:UInt32, alias2:UInt64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; @@ -544,9 +544,9 @@ mod tests { )? .build()?; // Should work - let expected = "Projection: test.a, sum(alias2) AS sum(test.c), MAX(alias3) AS MAX(test.c), count(alias1) AS count(DISTINCT test.b) [a:UInt32, sum(test.c):UInt64;N, MAX(test.c):UInt32;N, count(DISTINCT test.b):Int64]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[sum(alias2), MAX(alias3), count(alias1)]] [a:UInt32, sum(alias2):UInt64;N, MAX(alias3):UInt32;N, count(alias1):Int64]\ - \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[sum(test.c) AS alias2, MAX(test.c) AS alias3]] [a:UInt32, alias1:UInt32, alias2:UInt64;N, alias3:UInt32;N]\ + let expected = "Projection: test.a, sum(alias2) AS sum(test.c), max(alias3) AS max(test.c), count(alias1) AS count(DISTINCT test.b) [a:UInt32, sum(test.c):UInt64;N, max(test.c):UInt32;N, count(DISTINCT test.b):Int64]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[sum(alias2), max(alias3), count(alias1)]] [a:UInt32, sum(alias2):UInt64;N, max(alias3):UInt32;N, count(alias1):Int64]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[sum(test.c) AS alias2, max(test.c) AS alias3]] [a:UInt32, alias1:UInt32, alias2:UInt64;N, alias3:UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -563,9 +563,9 @@ mod tests { )? .build()?; // Should work - let expected = "Projection: test.c, MIN(alias2) AS MIN(test.a), count(alias1) AS count(DISTINCT test.b) [c:UInt32, MIN(test.a):UInt32;N, count(DISTINCT test.b):Int64]\ - \n Aggregate: groupBy=[[test.c]], aggr=[[MIN(alias2), count(alias1)]] [c:UInt32, MIN(alias2):UInt32;N, count(alias1):Int64]\ - \n Aggregate: groupBy=[[test.c, test.b AS alias1]], aggr=[[MIN(test.a) AS alias2]] [c:UInt32, alias1:UInt32, alias2:UInt32;N]\ + let expected = "Projection: test.c, min(alias2) AS min(test.a), count(alias1) AS count(DISTINCT test.b) [c:UInt32, min(test.a):UInt32;N, count(DISTINCT test.b):Int64]\ + \n Aggregate: groupBy=[[test.c]], aggr=[[min(alias2), count(alias1)]] [c:UInt32, min(alias2):UInt32;N, count(alias1):Int64]\ + \n Aggregate: groupBy=[[test.c, test.b AS alias1]], aggr=[[min(test.a) AS alias2]] [c:UInt32, alias1:UInt32, alias2:UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 0760986a87c6..b477a815bf80 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -18,7 +18,6 @@ pub(crate) mod groups_accumulator; pub(crate) mod stats; -pub mod moving_min_max; pub mod utils { pub use datafusion_physical_expr_common::aggregate::utils::{ adjust_output_array, down_cast_any_ref, get_accum_scalar_values_as_arrays, diff --git a/datafusion/physical-expr/src/aggregate/moving_min_max.rs b/datafusion/physical-expr/src/aggregate/moving_min_max.rs deleted file mode 100644 index c4fb07679747..000000000000 --- a/datafusion/physical-expr/src/aggregate/moving_min_max.rs +++ /dev/null @@ -1,335 +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. - -// The implementation is taken from https://github.com/spebern/moving_min_max/blob/master/src/lib.rs. - -//! Keep track of the minimum or maximum value in a sliding window. -//! -//! `moving min max` provides one data structure for keeping track of the -//! minimum value and one for keeping track of the maximum value in a sliding -//! window. -//! -//! Each element is stored with the current min/max. One stack to push and another one for pop. If pop stack is empty, -//! push to this stack all elements popped from first stack while updating their current min/max. Now pop from -//! the second stack (MovingMin/Max struct works as a queue). To find the minimum element of the queue, -//! look at the smallest/largest two elements of the individual stacks, then take the minimum of those two values. -//! -//! The complexity of the operations are -//! - O(1) for getting the minimum/maximum -//! - O(1) for push -//! - amortized O(1) for pop - -/// ``` -/// # use datafusion_physical_expr::aggregate::moving_min_max::MovingMin; -/// let mut moving_min = MovingMin::::new(); -/// moving_min.push(2); -/// moving_min.push(1); -/// moving_min.push(3); -/// -/// assert_eq!(moving_min.min(), Some(&1)); -/// assert_eq!(moving_min.pop(), Some(2)); -/// -/// assert_eq!(moving_min.min(), Some(&1)); -/// assert_eq!(moving_min.pop(), Some(1)); -/// -/// assert_eq!(moving_min.min(), Some(&3)); -/// assert_eq!(moving_min.pop(), Some(3)); -/// -/// assert_eq!(moving_min.min(), None); -/// assert_eq!(moving_min.pop(), None); -/// ``` -#[derive(Debug)] -pub struct MovingMin { - push_stack: Vec<(T, T)>, - pop_stack: Vec<(T, T)>, -} - -impl Default for MovingMin { - fn default() -> Self { - Self { - push_stack: Vec::new(), - pop_stack: Vec::new(), - } - } -} - -impl MovingMin { - /// Creates a new `MovingMin` to keep track of the minimum in a sliding - /// window. - #[inline] - pub fn new() -> Self { - Self::default() - } - - /// Creates a new `MovingMin` to keep track of the minimum in a sliding - /// window with `capacity` allocated slots. - #[inline] - pub fn with_capacity(capacity: usize) -> Self { - Self { - push_stack: Vec::with_capacity(capacity), - pop_stack: Vec::with_capacity(capacity), - } - } - - /// Returns the minimum of the sliding window or `None` if the window is - /// empty. - #[inline] - pub fn min(&self) -> Option<&T> { - match (self.push_stack.last(), self.pop_stack.last()) { - (None, None) => None, - (Some((_, min)), None) => Some(min), - (None, Some((_, min))) => Some(min), - (Some((_, a)), Some((_, b))) => Some(if a < b { a } else { b }), - } - } - - /// Pushes a new element into the sliding window. - #[inline] - pub fn push(&mut self, val: T) { - self.push_stack.push(match self.push_stack.last() { - Some((_, min)) => { - if val > *min { - (val, min.clone()) - } else { - (val.clone(), val) - } - } - None => (val.clone(), val), - }); - } - - /// Removes and returns the last value of the sliding window. - #[inline] - pub fn pop(&mut self) -> Option { - if self.pop_stack.is_empty() { - match self.push_stack.pop() { - Some((val, _)) => { - let mut last = (val.clone(), val); - self.pop_stack.push(last.clone()); - while let Some((val, _)) = self.push_stack.pop() { - let min = if last.1 < val { - last.1.clone() - } else { - val.clone() - }; - last = (val.clone(), min); - self.pop_stack.push(last.clone()); - } - } - None => return None, - } - } - self.pop_stack.pop().map(|(val, _)| val) - } - - /// Returns the number of elements stored in the sliding window. - #[inline] - pub fn len(&self) -> usize { - self.push_stack.len() + self.pop_stack.len() - } - - /// Returns `true` if the moving window contains no elements. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } -} -/// ``` -/// # use datafusion_physical_expr::aggregate::moving_min_max::MovingMax; -/// let mut moving_max = MovingMax::::new(); -/// moving_max.push(2); -/// moving_max.push(3); -/// moving_max.push(1); -/// -/// assert_eq!(moving_max.max(), Some(&3)); -/// assert_eq!(moving_max.pop(), Some(2)); -/// -/// assert_eq!(moving_max.max(), Some(&3)); -/// assert_eq!(moving_max.pop(), Some(3)); -/// -/// assert_eq!(moving_max.max(), Some(&1)); -/// assert_eq!(moving_max.pop(), Some(1)); -/// -/// assert_eq!(moving_max.max(), None); -/// assert_eq!(moving_max.pop(), None); -/// ``` -#[derive(Debug)] -pub struct MovingMax { - push_stack: Vec<(T, T)>, - pop_stack: Vec<(T, T)>, -} - -impl Default for MovingMax { - fn default() -> Self { - Self { - push_stack: Vec::new(), - pop_stack: Vec::new(), - } - } -} - -impl MovingMax { - /// Creates a new `MovingMax` to keep track of the maximum in a sliding window. - #[inline] - pub fn new() -> Self { - Self::default() - } - - /// Creates a new `MovingMax` to keep track of the maximum in a sliding window with - /// `capacity` allocated slots. - #[inline] - pub fn with_capacity(capacity: usize) -> Self { - Self { - push_stack: Vec::with_capacity(capacity), - pop_stack: Vec::with_capacity(capacity), - } - } - - /// Returns the maximum of the sliding window or `None` if the window is empty. - #[inline] - pub fn max(&self) -> Option<&T> { - match (self.push_stack.last(), self.pop_stack.last()) { - (None, None) => None, - (Some((_, max)), None) => Some(max), - (None, Some((_, max))) => Some(max), - (Some((_, a)), Some((_, b))) => Some(if a > b { a } else { b }), - } - } - - /// Pushes a new element into the sliding window. - #[inline] - pub fn push(&mut self, val: T) { - self.push_stack.push(match self.push_stack.last() { - Some((_, max)) => { - if val < *max { - (val, max.clone()) - } else { - (val.clone(), val) - } - } - None => (val.clone(), val), - }); - } - - /// Removes and returns the last value of the sliding window. - #[inline] - pub fn pop(&mut self) -> Option { - if self.pop_stack.is_empty() { - match self.push_stack.pop() { - Some((val, _)) => { - let mut last = (val.clone(), val); - self.pop_stack.push(last.clone()); - while let Some((val, _)) = self.push_stack.pop() { - let max = if last.1 > val { - last.1.clone() - } else { - val.clone() - }; - last = (val.clone(), max); - self.pop_stack.push(last.clone()); - } - } - None => return None, - } - } - self.pop_stack.pop().map(|(val, _)| val) - } - - /// Returns the number of elements stored in the sliding window. - #[inline] - pub fn len(&self) -> usize { - self.push_stack.len() + self.pop_stack.len() - } - - /// Returns `true` if the moving window contains no elements. - #[inline] - pub fn is_empty(&self) -> bool { - self.len() == 0 - } -} - -#[cfg(test)] -mod tests { - use super::*; - use datafusion_common::Result; - use rand::Rng; - - fn get_random_vec_i32(len: usize) -> Vec { - let mut rng = rand::thread_rng(); - let mut input = Vec::with_capacity(len); - for _i in 0..len { - input.push(rng.gen_range(0..100)); - } - input - } - - fn moving_min_i32(len: usize, n_sliding_window: usize) -> Result<()> { - let data = get_random_vec_i32(len); - let mut expected = Vec::with_capacity(len); - let mut moving_min = MovingMin::::new(); - let mut res = Vec::with_capacity(len); - for i in 0..len { - let start = i.saturating_sub(n_sliding_window); - expected.push(*data[start..i + 1].iter().min().unwrap()); - - moving_min.push(data[i]); - if i > n_sliding_window { - moving_min.pop(); - } - res.push(*moving_min.min().unwrap()); - } - assert_eq!(res, expected); - Ok(()) - } - - fn moving_max_i32(len: usize, n_sliding_window: usize) -> Result<()> { - let data = get_random_vec_i32(len); - let mut expected = Vec::with_capacity(len); - let mut moving_max = MovingMax::::new(); - let mut res = Vec::with_capacity(len); - for i in 0..len { - let start = i.saturating_sub(n_sliding_window); - expected.push(*data[start..i + 1].iter().max().unwrap()); - - moving_max.push(data[i]); - if i > n_sliding_window { - moving_max.pop(); - } - res.push(*moving_max.max().unwrap()); - } - assert_eq!(res, expected); - Ok(()) - } - - #[test] - fn moving_min_tests() -> Result<()> { - moving_min_i32(100, 10)?; - moving_min_i32(100, 20)?; - moving_min_i32(100, 50)?; - moving_min_i32(100, 100)?; - Ok(()) - } - - #[test] - fn moving_max_tests() -> Result<()> { - moving_max_i32(100, 10)?; - moving_max_i32(100, 20)?; - moving_max_i32(100, 50)?; - moving_max_i32(100, 100)?; - Ok(()) - } -} diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index b65f976a2251..8f9f1dd78f93 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -295,7 +295,7 @@ fn roundtrip_statement_with_dialect() -> Result<()> { sql: "SELECT string_count FROM ( SELECT j1_id, - MIN(j2_string) + min(j2_string) FROM j1 LEFT OUTER JOIN j2 ON j1_id = j2_id @@ -303,7 +303,7 @@ fn roundtrip_statement_with_dialect() -> Result<()> { j1_id ) AS agg (id, string_count) ", - expected: r#"SELECT agg.string_count FROM (SELECT j1.j1_id, MIN(j2.j2_string) FROM j1 LEFT JOIN j2 ON (j1.j1_id = j2.j2_id) GROUP BY j1.j1_id) AS agg (id, string_count)"#, + expected: r#"SELECT agg.string_count FROM (SELECT j1.j1_id, min(j2.j2_string) FROM j1 LEFT JOIN j2 ON (j1.j1_id = j2.j2_id) GROUP BY j1.j1_id) AS agg (id, string_count)"#, parser_dialect: Box::new(GenericDialect {}), unparser_dialect: Box::new(UnparserDefaultDialect {}), }, diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index e61c29f1c80d..8a5510eb69f3 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -1016,7 +1016,7 @@ fn select_with_having_refers_to_invalid_column() { HAVING first_name = 'M'"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: person.id, MAX(person.age)", + "Error during planning: HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: person.id, max(person.age)", err.strip_backtrace() ); } @@ -1040,7 +1040,7 @@ fn select_with_having_with_aggregate_not_in_select() { HAVING MAX(age) > 100"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projection references non-aggregate values: Expression person.first_name could not be resolved from available columns: MAX(person.age)", + "Error during planning: Projection references non-aggregate values: Expression person.first_name could not be resolved from available columns: max(person.age)", err.strip_backtrace() ); } @@ -1050,21 +1050,21 @@ fn select_aggregate_with_having_that_reuses_aggregate() { let sql = "SELECT MAX(age) FROM person HAVING MAX(age) < 30"; - let expected = "Projection: MAX(person.age)\ - \n Filter: MAX(person.age) < Int64(30)\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: max(person.age)\ + \n Filter: max(person.age) < Int64(30)\ + \n Aggregate: groupBy=[[]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } #[test] fn select_aggregate_with_having_with_aggregate_not_in_select() { - let sql = "SELECT MAX(age) + let sql = "SELECT max(age) FROM person - HAVING MAX(first_name) > 'M'"; - let expected = "Projection: MAX(person.age)\ - \n Filter: MAX(person.first_name) > Utf8(\"M\")\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(person.age), MAX(person.first_name)]]\ + HAVING max(first_name) > 'M'"; + let expected = "Projection: max(person.age)\ + \n Filter: max(person.first_name) > Utf8(\"M\")\ + \n Aggregate: groupBy=[[]], aggr=[[max(person.age), max(person.first_name)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1087,21 +1087,21 @@ fn select_aggregate_aliased_with_having_referencing_aggregate_by_its_alias() { FROM person HAVING max_age < 30"; // FIXME: add test for having in execution - let expected = "Projection: MAX(person.age) AS max_age\ - \n Filter: MAX(person.age) < Int64(30)\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: max(person.age) AS max_age\ + \n Filter: max(person.age) < Int64(30)\ + \n Aggregate: groupBy=[[]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } #[test] fn select_aggregate_aliased_with_having_that_reuses_aggregate_but_not_by_its_alias() { - let sql = "SELECT MAX(age) as max_age + let sql = "SELECT max(age) as max_age FROM person - HAVING MAX(age) < 30"; - let expected = "Projection: MAX(person.age) AS max_age\ - \n Filter: MAX(person.age) < Int64(30)\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(person.age)]]\ + HAVING max(age) < 30"; + let expected = "Projection: max(person.age) AS max_age\ + \n Filter: max(person.age) < Int64(30)\ + \n Aggregate: groupBy=[[]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1112,23 +1112,23 @@ fn select_aggregate_with_group_by_with_having() { FROM person GROUP BY first_name HAVING first_name = 'M'"; - let expected = "Projection: person.first_name, MAX(person.age)\ + let expected = "Projection: person.first_name, max(person.age)\ \n Filter: person.first_name = Utf8(\"M\")\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } #[test] fn select_aggregate_with_group_by_with_having_and_where() { - let sql = "SELECT first_name, MAX(age) + let sql = "SELECT first_name, max(age) FROM person WHERE id > 5 GROUP BY first_name HAVING MAX(age) < 100"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) < Int64(100)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) < Int64(100)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n Filter: person.id > Int64(5)\ \n TableScan: person"; quick_test(sql, expected); @@ -1141,9 +1141,9 @@ fn select_aggregate_with_group_by_with_having_and_where_filtering_on_aggregate_c WHERE id > 5 AND age > 18 GROUP BY first_name HAVING MAX(age) < 100"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) < Int64(100)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) < Int64(100)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n Filter: person.id > Int64(5) AND person.age > Int64(18)\ \n TableScan: person"; quick_test(sql, expected); @@ -1155,9 +1155,9 @@ fn select_aggregate_with_group_by_with_having_using_column_by_alias() { FROM person GROUP BY first_name HAVING MAX(age) > 2 AND fn = 'M'"; - let expected = "Projection: person.first_name AS fn, MAX(person.age)\ - \n Filter: MAX(person.age) > Int64(2) AND person.first_name = Utf8(\"M\")\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name AS fn, max(person.age)\ + \n Filter: max(person.age) > Int64(2) AND person.first_name = Utf8(\"M\")\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1169,9 +1169,9 @@ fn select_aggregate_with_group_by_with_having_using_columns_with_and_without_the FROM person GROUP BY first_name HAVING MAX(age) > 2 AND max_age < 5 AND first_name = 'M' AND fn = 'N'"; - let expected = "Projection: person.first_name AS fn, MAX(person.age) AS max_age\ - \n Filter: MAX(person.age) > Int64(2) AND MAX(person.age) < Int64(5) AND person.first_name = Utf8(\"M\") AND person.first_name = Utf8(\"N\")\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name AS fn, max(person.age) AS max_age\ + \n Filter: max(person.age) > Int64(2) AND max(person.age) < Int64(5) AND person.first_name = Utf8(\"M\") AND person.first_name = Utf8(\"N\")\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1182,9 +1182,9 @@ fn select_aggregate_with_group_by_with_having_that_reuses_aggregate() { FROM person GROUP BY first_name HAVING MAX(age) > 100"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) > Int64(100)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) > Int64(100)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1197,7 +1197,7 @@ fn select_aggregate_with_group_by_with_having_referencing_column_not_in_group_by HAVING MAX(age) > 10 AND last_name = 'M'"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: HAVING clause references non-aggregate values: Expression person.last_name could not be resolved from available columns: person.first_name, MAX(person.age)", + "Error during planning: HAVING clause references non-aggregate values: Expression person.last_name could not be resolved from available columns: person.first_name, max(person.age)", err.strip_backtrace() ); } @@ -1208,9 +1208,9 @@ fn select_aggregate_with_group_by_with_having_that_reuses_aggregate_multiple_tim FROM person GROUP BY first_name HAVING MAX(age) > 100 AND MAX(age) < 200"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) > Int64(100) AND MAX(person.age) < Int64(200)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) > Int64(100) AND max(person.age) < Int64(200)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1221,9 +1221,9 @@ fn select_aggregate_with_group_by_with_having_using_aggregate_not_in_select() { FROM person GROUP BY first_name HAVING MAX(age) > 100 AND MIN(id) < 50"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) > Int64(100) AND MIN(person.id) < Int64(50)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age), MIN(person.id)]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) > Int64(100) AND min(person.id) < Int64(50)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age), min(person.id)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1235,9 +1235,9 @@ fn select_aggregate_aliased_with_group_by_with_having_referencing_aggregate_by_i FROM person GROUP BY first_name HAVING max_age > 100"; - let expected = "Projection: person.first_name, MAX(person.age) AS max_age\ - \n Filter: MAX(person.age) > Int64(100)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name, max(person.age) AS max_age\ + \n Filter: max(person.age) > Int64(100)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1249,9 +1249,9 @@ fn select_aggregate_compound_aliased_with_group_by_with_having_referencing_compo FROM person GROUP BY first_name HAVING max_age_plus_one > 100"; - let expected = "Projection: person.first_name, MAX(person.age) + Int64(1) AS max_age_plus_one\ - \n Filter: MAX(person.age) + Int64(1) > Int64(100)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: person.first_name, max(person.age) + Int64(1) AS max_age_plus_one\ + \n Filter: max(person.age) + Int64(1) > Int64(100)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1263,9 +1263,9 @@ fn select_aggregate_with_group_by_with_having_using_derived_column_aggregate_not FROM person GROUP BY first_name HAVING MAX(age) > 100 AND MIN(id - 2) < 50"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) > Int64(100) AND MIN(person.id - Int64(2)) < Int64(50)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age), MIN(person.id - Int64(2))]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) > Int64(100) AND min(person.id - Int64(2)) < Int64(50)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age), min(person.id - Int64(2))]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1276,9 +1276,9 @@ fn select_aggregate_with_group_by_with_having_using_count_star_not_in_select() { FROM person GROUP BY first_name HAVING MAX(age) > 100 AND count(*) < 50"; - let expected = "Projection: person.first_name, MAX(person.age)\ - \n Filter: MAX(person.age) > Int64(100) AND count(*) < Int64(50)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.age), count(*)]]\ + let expected = "Projection: person.first_name, max(person.age)\ + \n Filter: max(person.age) > Int64(100) AND count(*) < Int64(50)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.age), count(*)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1321,8 +1321,8 @@ fn select_wildcard_with_groupby() { fn select_simple_aggregate() { quick_test( "SELECT MIN(age) FROM person", - "Projection: MIN(person.age)\ - \n Aggregate: groupBy=[[]], aggr=[[MIN(person.age)]]\ + "Projection: min(person.age)\ + \n Aggregate: groupBy=[[]], aggr=[[min(person.age)]]\ \n TableScan: person", ); } @@ -1349,7 +1349,7 @@ fn select_simple_aggregate_repeated_aggregate() { let sql = "SELECT MIN(age), MIN(age) FROM person"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projections require unique expression names but the expression \"MIN(person.age)\" at position 0 and \"MIN(person.age)\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.", + "Error during planning: Projections require unique expression names but the expression \"min(person.age)\" at position 0 and \"min(person.age)\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.", err.strip_backtrace() ); } @@ -1358,8 +1358,8 @@ fn select_simple_aggregate_repeated_aggregate() { fn select_simple_aggregate_repeated_aggregate_with_single_alias() { quick_test( "SELECT MIN(age), MIN(age) AS a FROM person", - "Projection: MIN(person.age), MIN(person.age) AS a\ - \n Aggregate: groupBy=[[]], aggr=[[MIN(person.age)]]\ + "Projection: min(person.age), min(person.age) AS a\ + \n Aggregate: groupBy=[[]], aggr=[[min(person.age)]]\ \n TableScan: person", ); } @@ -1368,8 +1368,8 @@ fn select_simple_aggregate_repeated_aggregate_with_single_alias() { fn select_simple_aggregate_repeated_aggregate_with_unique_aliases() { quick_test( "SELECT MIN(age) AS a, MIN(age) AS b FROM person", - "Projection: MIN(person.age) AS a, MIN(person.age) AS b\ - \n Aggregate: groupBy=[[]], aggr=[[MIN(person.age)]]\ + "Projection: min(person.age) AS a, min(person.age) AS b\ + \n Aggregate: groupBy=[[]], aggr=[[min(person.age)]]\ \n TableScan: person", ); } @@ -1390,7 +1390,7 @@ fn select_simple_aggregate_repeated_aggregate_with_repeated_aliases() { let sql = "SELECT MIN(age) AS a, MIN(age) AS a FROM person"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projections require unique expression names but the expression \"MIN(person.age) AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.", + "Error during planning: Projections require unique expression names but the expression \"min(person.age) AS a\" at position 0 and \"min(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.", err.strip_backtrace() ); } @@ -1399,8 +1399,8 @@ fn select_simple_aggregate_repeated_aggregate_with_repeated_aliases() { fn select_simple_aggregate_with_groupby() { quick_test( "SELECT state, MIN(age), MAX(age) FROM person GROUP BY state", - "Projection: person.state, MIN(person.age), MAX(person.age)\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age), MAX(person.age)]]\ + "Projection: person.state, min(person.age), max(person.age)\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age), max(person.age)]]\ \n TableScan: person", ); } @@ -1409,8 +1409,8 @@ fn select_simple_aggregate_with_groupby() { fn select_simple_aggregate_with_groupby_with_aliases() { quick_test( "SELECT state AS a, MIN(age) AS b FROM person GROUP BY state", - "Projection: person.state AS a, MIN(person.age) AS b\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age)]]\ + "Projection: person.state AS a, min(person.age) AS b\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age)]]\ \n TableScan: person", ); } @@ -1420,7 +1420,7 @@ fn select_simple_aggregate_with_groupby_with_aliases_repeated() { let sql = "SELECT state AS a, MIN(age) AS a FROM person GROUP BY state"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projections require unique expression names but the expression \"person.state AS a\" at position 0 and \"MIN(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.", + "Error during planning: Projections require unique expression names but the expression \"person.state AS a\" at position 0 and \"min(person.age) AS a\" at position 1 have the same name. Consider aliasing (\"AS\") one of them.", err.strip_backtrace() ); } @@ -1429,8 +1429,8 @@ fn select_simple_aggregate_with_groupby_with_aliases_repeated() { fn select_simple_aggregate_with_groupby_column_unselected() { quick_test( "SELECT MIN(age), MAX(age) FROM person GROUP BY state", - "Projection: MIN(person.age), MAX(person.age)\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age), MAX(person.age)]]\ + "Projection: min(person.age), max(person.age)\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age), max(person.age)]]\ \n TableScan: person", ); } @@ -1514,8 +1514,8 @@ fn recursive_ctes_disabled() { fn select_simple_aggregate_with_groupby_and_column_is_in_aggregate_and_groupby() { quick_test( "SELECT MAX(first_name) FROM person GROUP BY first_name", - "Projection: MAX(person.first_name)\ - \n Aggregate: groupBy=[[person.first_name]], aggr=[[MAX(person.first_name)]]\ + "Projection: max(person.first_name)\ + \n Aggregate: groupBy=[[person.first_name]], aggr=[[max(person.first_name)]]\ \n TableScan: person", ); } @@ -1557,8 +1557,8 @@ fn select_simple_aggregate_with_groupby_position_out_of_range() { fn select_simple_aggregate_with_groupby_can_use_alias() { quick_test( "SELECT state AS a, MIN(age) AS b FROM person GROUP BY a", - "Projection: person.state AS a, MIN(person.age) AS b\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age)]]\ + "Projection: person.state AS a, min(person.age) AS b\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age)]]\ \n TableScan: person", ); } @@ -1568,7 +1568,7 @@ fn select_simple_aggregate_with_groupby_aggregate_repeated() { let sql = "SELECT state, MIN(age), MIN(age) FROM person GROUP BY state"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projections require unique expression names but the expression \"MIN(person.age)\" at position 1 and \"MIN(person.age)\" at position 2 have the same name. Consider aliasing (\"AS\") one of them.", + "Error during planning: Projections require unique expression names but the expression \"min(person.age)\" at position 1 and \"min(person.age)\" at position 2 have the same name. Consider aliasing (\"AS\") one of them.", err.strip_backtrace() ); } @@ -1577,8 +1577,8 @@ fn select_simple_aggregate_with_groupby_aggregate_repeated() { fn select_simple_aggregate_with_groupby_aggregate_repeated_and_one_has_alias() { quick_test( "SELECT state, MIN(age), MIN(age) AS ma FROM person GROUP BY state", - "Projection: person.state, MIN(person.age), MIN(person.age) AS ma\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age)]]\ + "Projection: person.state, min(person.age), min(person.age) AS ma\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age)]]\ \n TableScan: person", ) } @@ -1587,8 +1587,8 @@ fn select_simple_aggregate_with_groupby_aggregate_repeated_and_one_has_alias() { fn select_simple_aggregate_with_groupby_non_column_expression_unselected() { quick_test( "SELECT MIN(first_name) FROM person GROUP BY age + 1", - "Projection: MIN(person.first_name)\ - \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[MIN(person.first_name)]]\ + "Projection: min(person.first_name)\ + \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[min(person.first_name)]]\ \n TableScan: person", ); } @@ -1597,14 +1597,14 @@ fn select_simple_aggregate_with_groupby_non_column_expression_unselected() { fn select_simple_aggregate_with_groupby_non_column_expression_selected_and_resolvable() { quick_test( "SELECT age + 1, MIN(first_name) FROM person GROUP BY age + 1", - "Projection: person.age + Int64(1), MIN(person.first_name)\ - \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[MIN(person.first_name)]]\ + "Projection: person.age + Int64(1), min(person.first_name)\ + \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[min(person.first_name)]]\ \n TableScan: person", ); quick_test( "SELECT MIN(first_name), age + 1 FROM person GROUP BY age + 1", - "Projection: MIN(person.first_name), person.age + Int64(1)\ - \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[MIN(person.first_name)]]\ + "Projection: min(person.first_name), person.age + Int64(1)\ + \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[min(person.first_name)]]\ \n TableScan: person", ); } @@ -1613,8 +1613,8 @@ fn select_simple_aggregate_with_groupby_non_column_expression_selected_and_resol fn select_simple_aggregate_with_groupby_non_column_expression_nested_and_resolvable() { quick_test( "SELECT ((age + 1) / 2) * (age + 1), MIN(first_name) FROM person GROUP BY age + 1", - "Projection: person.age + Int64(1) / Int64(2) * person.age + Int64(1), MIN(person.first_name)\ - \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[MIN(person.first_name)]]\ + "Projection: person.age + Int64(1) / Int64(2) * person.age + Int64(1), min(person.first_name)\ + \n Aggregate: groupBy=[[person.age + Int64(1)]], aggr=[[min(person.first_name)]]\ \n TableScan: person", ); } @@ -1626,7 +1626,7 @@ fn select_simple_aggregate_with_groupby_non_column_expression_nested_and_not_res let sql = "SELECT ((age + 1) / 2) * (age + 9), MIN(first_name) FROM person GROUP BY age + 1"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)", + "Error during planning: Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), min(person.first_name)", err.strip_backtrace() ); } @@ -1636,7 +1636,7 @@ fn select_simple_aggregate_with_groupby_non_column_expression_and_its_column_sel let sql = "SELECT age, MIN(first_name) FROM person GROUP BY age + 1"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), MIN(person.first_name)", + "Error during planning: Projection references non-aggregate values: Expression person.age could not be resolved from available columns: person.age + Int64(1), min(person.first_name)", err.strip_backtrace() ); } @@ -1645,8 +1645,8 @@ fn select_simple_aggregate_with_groupby_non_column_expression_and_its_column_sel fn select_simple_aggregate_nested_in_binary_expr_with_groupby() { quick_test( "SELECT state, MIN(age) < 10 FROM person GROUP BY state", - "Projection: person.state, MIN(person.age) < Int64(10)\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age)]]\ + "Projection: person.state, min(person.age) < Int64(10)\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age)]]\ \n TableScan: person", ); } @@ -1655,8 +1655,8 @@ fn select_simple_aggregate_nested_in_binary_expr_with_groupby() { fn select_simple_aggregate_and_nested_groupby_column() { quick_test( "SELECT age + 1, MAX(first_name) FROM person GROUP BY age", - "Projection: person.age + Int64(1), MAX(person.first_name)\ - \n Aggregate: groupBy=[[person.age]], aggr=[[MAX(person.first_name)]]\ + "Projection: person.age + Int64(1), max(person.first_name)\ + \n Aggregate: groupBy=[[person.age]], aggr=[[max(person.first_name)]]\ \n TableScan: person", ); } @@ -1665,8 +1665,8 @@ fn select_simple_aggregate_and_nested_groupby_column() { fn select_aggregate_compounded_with_groupby_column() { quick_test( "SELECT age + MIN(salary) FROM person GROUP BY age", - "Projection: person.age + MIN(person.salary)\ - \n Aggregate: groupBy=[[person.age]], aggr=[[MIN(person.salary)]]\ + "Projection: person.age + min(person.salary)\ + \n Aggregate: groupBy=[[person.age]], aggr=[[min(person.salary)]]\ \n TableScan: person", ); } @@ -1675,8 +1675,8 @@ fn select_aggregate_compounded_with_groupby_column() { fn select_aggregate_with_non_column_inner_expression_with_groupby() { quick_test( "SELECT state, MIN(age + 1) FROM person GROUP BY state", - "Projection: person.state, MIN(person.age + Int64(1))\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MIN(person.age + Int64(1))]]\ + "Projection: person.state, min(person.age + Int64(1))\ + \n Aggregate: groupBy=[[person.state]], aggr=[[min(person.age + Int64(1))]]\ \n TableScan: person", ); } @@ -1864,8 +1864,8 @@ fn select_group_by() { #[test] fn select_group_by_columns_not_in_select() { let sql = "SELECT MAX(age) FROM person GROUP BY state"; - let expected = "Projection: MAX(person.age)\ - \n Aggregate: groupBy=[[person.state]], aggr=[[MAX(person.age)]]\ + let expected = "Projection: max(person.age)\ + \n Aggregate: groupBy=[[person.state]], aggr=[[max(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); @@ -1895,8 +1895,8 @@ fn select_group_by_needs_projection() { #[test] fn select_7480_1() { let sql = "SELECT c1, MIN(c12) FROM aggregate_test_100 GROUP BY c1, c13"; - let expected = "Projection: aggregate_test_100.c1, MIN(aggregate_test_100.c12)\ - \n Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c13]], aggr=[[MIN(aggregate_test_100.c12)]]\ + let expected = "Projection: aggregate_test_100.c1, min(aggregate_test_100.c12)\ + \n Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c13]], aggr=[[min(aggregate_test_100.c12)]]\ \n TableScan: aggregate_test_100"; quick_test(sql, expected); } @@ -1906,7 +1906,7 @@ fn select_7480_2() { let sql = "SELECT c1, c13, MIN(c12) FROM aggregate_test_100 GROUP BY c1"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Projection references non-aggregate values: Expression aggregate_test_100.c13 could not be resolved from available columns: aggregate_test_100.c1, MIN(aggregate_test_100.c12)", + "Error during planning: Projection references non-aggregate values: Expression aggregate_test_100.c13 could not be resolved from available columns: aggregate_test_100.c1, min(aggregate_test_100.c12)", err.strip_backtrace() ); } @@ -2319,8 +2319,8 @@ fn union_with_incompatible_data_types() { fn empty_over() { let sql = "SELECT order_id, MAX(order_id) OVER () from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id, max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2329,8 +2329,8 @@ fn empty_over() { fn empty_over_with_alias() { let sql = "SELECT order_id oid, MAX(order_id) OVER () max_oid from orders"; let expected = "\ - Projection: orders.order_id AS oid, MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_oid\ - \n WindowAggr: windowExpr=[[MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id AS oid, max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_oid\ + \n WindowAggr: windowExpr=[[max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2339,8 +2339,8 @@ fn empty_over_with_alias() { fn empty_over_dup_with_alias() { let sql = "SELECT order_id oid, MAX(order_id) OVER () max_oid, MAX(order_id) OVER () max_oid_dup from orders"; let expected = "\ - Projection: orders.order_id AS oid, MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_oid, MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_oid_dup\ - \n WindowAggr: windowExpr=[[MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id AS oid, max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_oid, max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max_oid_dup\ + \n WindowAggr: windowExpr=[[max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2349,9 +2349,9 @@ fn empty_over_dup_with_alias() { fn empty_over_dup_with_different_sort() { let sql = "SELECT order_id oid, MAX(order_id) OVER (), MAX(order_id) OVER (ORDER BY order_id) from orders"; let expected = "\ - Projection: orders.order_id AS oid, MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MAX(orders.order_id) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n WindowAggr: windowExpr=[[MAX(orders.order_id) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id AS oid, max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, max(orders.order_id) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.order_id) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + \n WindowAggr: windowExpr=[[max(orders.order_id) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2360,8 +2360,8 @@ fn empty_over_dup_with_different_sort() { fn empty_over_plus() { let sql = "SELECT order_id, MAX(qty * 1.1) OVER () from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty * Float64(1.1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(orders.qty * Float64(1.1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id, max(orders.qty * Float64(1.1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[max(orders.qty * Float64(1.1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2370,8 +2370,8 @@ fn empty_over_plus() { fn empty_over_multiple() { let sql = "SELECT order_id, MAX(qty) OVER (), min(qty) over (), avg(qty) OVER () from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, avg(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, avg(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id, max(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, avg(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[max(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, avg(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2389,8 +2389,8 @@ fn empty_over_multiple() { fn over_partition_by() { let sql = "SELECT order_id, MAX(qty) OVER (PARTITION BY order_id) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id, max(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[max(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2411,9 +2411,9 @@ fn over_partition_by() { fn over_order_by() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY order_id), MIN(qty) OVER (ORDER BY order_id DESC) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2422,9 +2422,9 @@ fn over_order_by() { fn over_order_by_with_window_frame_double_end() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY order_id ROWS BETWEEN 3 PRECEDING and 3 FOLLOWING), MIN(qty) OVER (ORDER BY order_id DESC) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING, MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING, min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2433,9 +2433,9 @@ fn over_order_by_with_window_frame_double_end() { fn over_order_by_with_window_frame_single_end() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY order_id ROWS 3 PRECEDING), MIN(qty) OVER (ORDER BY order_id DESC) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND CURRENT ROW, MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND CURRENT ROW, min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] ROWS BETWEEN 3 PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2444,9 +2444,9 @@ fn over_order_by_with_window_frame_single_end() { fn over_order_by_with_window_frame_single_end_groups() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY order_id GROUPS 3 PRECEDING), MIN(qty) OVER (ORDER BY order_id DESC) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] GROUPS BETWEEN 3 PRECEDING AND CURRENT ROW, MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] GROUPS BETWEEN 3 PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] GROUPS BETWEEN 3 PRECEDING AND CURRENT ROW, min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] GROUPS BETWEEN 3 PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2467,9 +2467,9 @@ fn over_order_by_with_window_frame_single_end_groups() { fn over_order_by_two_sort_keys() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY order_id), MIN(qty) OVER (ORDER BY (order_id + 1)) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(orders.qty) ORDER BY [orders.order_id + Int64(1) ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id + Int64(1) ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, min(orders.qty) ORDER BY [orders.order_id + Int64(1) ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id + Int64(1) ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2491,10 +2491,10 @@ fn over_order_by_two_sort_keys() { fn over_order_by_sort_keys_sorting() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY qty, order_id), sum(qty) OVER (), MIN(qty) OVER (ORDER BY order_id, qty) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ \n WindowAggr: windowExpr=[[sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2514,10 +2514,10 @@ fn over_order_by_sort_keys_sorting() { fn over_order_by_sort_keys_sorting_prefix_compacting() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY order_id), sum(qty) OVER (), MIN(qty) OVER (ORDER BY order_id, qty) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + Projection: orders.order_id, max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ \n WindowAggr: windowExpr=[[sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2543,10 +2543,10 @@ fn over_order_by_sort_keys_sorting_global_order_compacting() { let sql = "SELECT order_id, MAX(qty) OVER (ORDER BY qty, order_id), sum(qty) OVER (), MIN(qty) OVER (ORDER BY order_id, qty) from orders ORDER BY order_id"; let expected = "\ Sort: orders.order_id ASC NULLS LAST\ - \n Projection: orders.order_id, MAX(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n Projection: orders.order_id, max(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ \n WindowAggr: windowExpr=[[sum(orders.qty) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[max(orders.qty) ORDER BY [orders.qty ASC NULLS LAST, orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) ORDER BY [orders.order_id ASC NULLS LAST, orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2565,8 +2565,8 @@ fn over_partition_by_order_by() { let sql = "SELECT order_id, MAX(qty) OVER (PARTITION BY order_id ORDER BY qty) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2585,8 +2585,8 @@ fn over_partition_by_order_by_no_dup() { let sql = "SELECT order_id, MAX(qty) OVER (PARTITION BY order_id, qty ORDER BY qty) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2608,9 +2608,9 @@ fn over_partition_by_order_by_mix_up() { let sql = "SELECT order_id, MAX(qty) OVER (PARTITION BY order_id, qty ORDER BY qty), MIN(qty) OVER (PARTITION BY qty ORDER BY order_id) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(orders.qty) PARTITION BY [orders.qty] ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) PARTITION BY [orders.qty] ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, min(orders.qty) PARTITION BY [orders.qty] ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[min(orders.qty) PARTITION BY [orders.qty] ORDER BY [orders.order_id ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[max(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -2631,9 +2631,9 @@ fn over_partition_by_order_by_mix_up_prefix() { let sql = "SELECT order_id, MAX(qty) OVER (PARTITION BY order_id ORDER BY qty), MIN(qty) OVER (PARTITION BY order_id, qty ORDER BY price) from orders"; let expected = "\ - Projection: orders.order_id, MAX(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.price ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ - \n WindowAggr: windowExpr=[[MAX(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ - \n WindowAggr: windowExpr=[[MIN(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.price ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + Projection: orders.order_id, max(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, min(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.price ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\ + \n WindowAggr: windowExpr=[[max(orders.qty) PARTITION BY [orders.order_id] ORDER BY [orders.qty ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ + \n WindowAggr: windowExpr=[[min(orders.qty) PARTITION BY [orders.order_id, orders.qty] ORDER BY [orders.price ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]\ \n TableScan: orders"; quick_test(sql, expected); } @@ -3042,8 +3042,8 @@ fn scalar_subquery() { let expected = "Projection: p.id, ()\ \n Subquery:\ - \n Projection: MAX(person.id)\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(person.id)]]\ + \n Projection: max(person.id)\ + \n Aggregate: groupBy=[[]], aggr=[[max(person.id)]]\ \n Filter: person.last_name = outer_ref(p.last_name)\ \n TableScan: person\ \n SubqueryAlias: p\ @@ -4002,8 +4002,8 @@ fn test_prepare_statement_infer_types_subquery() { Projection: person.id, person.age Filter: person.age = () Subquery: - Projection: MAX(person.age) - Aggregate: groupBy=[[]], aggr=[[MAX(person.age)]] + Projection: max(person.age) + Aggregate: groupBy=[[]], aggr=[[max(person.age)]] Filter: person.id = $1 TableScan: person TableScan: person @@ -4023,8 +4023,8 @@ Projection: person.id, person.age Projection: person.id, person.age Filter: person.age = () Subquery: - Projection: MAX(person.age) - Aggregate: groupBy=[[]], aggr=[[MAX(person.age)]] + Projection: max(person.age) + Aggregate: groupBy=[[]], aggr=[[max(person.age)]] Filter: person.id = UInt32(10) TableScan: person TableScan: person @@ -4396,7 +4396,7 @@ fn test_field_not_found_window_function() { let qualified_sql = "SELECT order_id, MAX(qty) OVER (PARTITION BY orders.order_id) from orders"; - let expected = "Projection: orders.order_id, MAX(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\n WindowAggr: windowExpr=[[MAX(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\n TableScan: orders"; + let expected = "Projection: orders.order_id, max(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\n WindowAggr: windowExpr=[[max(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\n TableScan: orders"; quick_test(qualified_sql, expected); } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 6513258f879e..9625f02afbb4 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4645,16 +4645,16 @@ 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 +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)]] +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] +01)ProjectionExec: expr=[max(aggregate_test_100.c1)@2 as max(aggregate_test_100.c1), c2@0 as c2, c3@1 as c3] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[MAX(aggregate_test_100.c1)] +03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[max(aggregate_test_100.c1)] 04)------CoalescePartitionsExec -05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[MAX(aggregate_test_100.c1)] +05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)] 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 @@ -5280,20 +5280,20 @@ query TT EXPLAIN SELECT MIN(col0) FROM empty; ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[MIN(empty.col0)]] +01)Aggregate: groupBy=[[]], aggr=[[min(empty.col0)]] 02)--TableScan: empty projection=[col0] physical_plan -01)ProjectionExec: expr=[NULL as MIN(empty.col0)] +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)]] +01)Aggregate: groupBy=[[]], aggr=[[max(empty.col0)]] 02)--TableScan: empty projection=[col0] physical_plan -01)ProjectionExec: expr=[NULL as MAX(empty.col0)] +01)ProjectionExec: expr=[NULL as max(empty.col0)] 02)--PlaceholderRowExec statement ok diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 03555989a277..8e67f501dbd7 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -41,18 +41,18 @@ explain select trace_id, MAX(timestamp) from traces group by trace_id order by M ---- logical_plan 01)Limit: skip=0, fetch=4 -02)--Sort: MAX(traces.timestamp) DESC NULLS FIRST, fetch=4 -03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +02)--Sort: max(traces.timestamp) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +02)--SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] @@ -96,18 +96,18 @@ explain select trace_id, MAX(timestamp) from traces group by trace_id order by M ---- logical_plan 01)Limit: skip=0, fetch=4 -02)--Sort: MAX(traces.timestamp) DESC NULLS FIRST, fetch=4 -03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +02)--Sort: max(traces.timestamp) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] +02)--SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -115,18 +115,18 @@ explain select trace_id, MIN(timestamp) from traces group by trace_id order by M ---- logical_plan 01)Limit: skip=0, fetch=4 -02)--Sort: MIN(traces.timestamp) DESC NULLS FIRST, fetch=4 -03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MIN(traces.timestamp)]] +02)--Sort: min(traces.timestamp) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[min(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [MIN(traces.timestamp)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[MIN(traces.timestamp)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] +02)--SortPreservingMergeExec: [min(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -134,18 +134,18 @@ explain select trace_id, MAX(timestamp) from traces group by trace_id order by M ---- logical_plan 01)Limit: skip=0, fetch=4 -02)--Sort: MAX(traces.timestamp) ASC NULLS LAST, fetch=4 -03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +02)--Sort: max(traces.timestamp) ASC NULLS LAST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [MAX(traces.timestamp)@1 ASC NULLS LAST], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +02)--SortPreservingMergeExec: [max(traces.timestamp)@1 ASC NULLS LAST], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -154,21 +154,21 @@ explain select trace_id, MAX(timestamp) from traces group by trace_id order by t logical_plan 01)Limit: skip=0, fetch=4 02)--Sort: traces.trace_id ASC NULLS LAST, fetch=4 -03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 02)--SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 03)----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TI -select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; +select trace_id, max(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; ---- c 4 b 3 @@ -176,7 +176,7 @@ a 1 NULL 0 query TI -select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 4; +select trace_id, min(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 4; ---- b -2 a -1 @@ -184,21 +184,21 @@ NULL 0 c 2 query TI -select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 3; +select trace_id, max(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 3; ---- c 4 b 3 a 1 query TI -select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 3; +select trace_id, min(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 3; ---- b -2 a -1 NULL 0 query TII -select trace_id, other, MIN(timestamp) from traces group by trace_id, other order by MIN(timestamp) asc limit 4; +select trace_id, other, min(timestamp) from traces group by trace_id, other order by MIN(timestamp) asc limit 4; ---- b 0 -2 a -1 -1 @@ -206,7 +206,7 @@ NULL 0 0 a 1 1 query TII -select trace_id, MIN(other), MIN(timestamp) from traces group by trace_id order by MIN(timestamp), MIN(other) limit 4; +select trace_id, min(other), MIN(timestamp) from traces group by trace_id order by MIN(timestamp), MIN(other) limit 4; ---- b 0 -2 a -1 -1 diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index b08d329d4a86..1d5f9ba23d58 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2606,12 +2606,12 @@ query TT explain select min(a) filter (where a > 1) as x from t; ---- logical_plan -01)Projection: MIN(t.a) FILTER (WHERE t.a > Int64(1)) AS x -02)--Aggregate: groupBy=[[]], aggr=[[MIN(t.a) FILTER (WHERE t.a > Float32(1)) AS MIN(t.a) FILTER (WHERE t.a > Int64(1))]] +01)Projection: min(t.a) FILTER (WHERE t.a > Int64(1)) AS x +02)--Aggregate: groupBy=[[]], aggr=[[min(t.a) FILTER (WHERE t.a > Float32(1)) AS min(t.a) FILTER (WHERE t.a > Int64(1))]] 03)----TableScan: t projection=[a] physical_plan -01)ProjectionExec: expr=[MIN(t.a) FILTER (WHERE t.a > Int64(1))@0 as x] -02)--AggregateExec: mode=Single, gby=[], aggr=[MIN(t.a) FILTER (WHERE t.a > Int64(1))] +01)ProjectionExec: expr=[min(t.a) FILTER (WHERE t.a > Int64(1))@0 as x] +02)--AggregateExec: mode=Single, gby=[], aggr=[min(t.a) FILTER (WHERE t.a > Int64(1))] 03)----MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index a97c979c43a3..bd096f61fb5d 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4169,33 +4169,33 @@ query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT x) FROM t1 GROUP BY y; ---- logical_plan -01)Projection: sum(DISTINCT t1.x), MAX(DISTINCT t1.x) -02)--Aggregate: groupBy=[[t1.y]], aggr=[[sum(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] +01)Projection: sum(DISTINCT t1.x), max(DISTINCT t1.x) +02)--Aggregate: groupBy=[[t1.y]], aggr=[[sum(DISTINCT CAST(t1.x AS Float64)), max(DISTINCT t1.x)]] 03)----TableScan: t1 projection=[x, y] physical_plan -01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] -02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), MAX(DISTINCT t1.x)] +01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), max(DISTINCT t1.x)@2 as max(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), MAX(DISTINCT t1.x)] +06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 07)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; ---- logical_plan -01)Projection: sum(alias1) AS sum(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) -02)--Aggregate: groupBy=[[t1.y]], aggr=[[sum(alias1), MAX(alias1)]] +01)Projection: sum(alias1) AS sum(DISTINCT t1.x), max(alias1) AS max(DISTINCT t1.x) +02)--Aggregate: groupBy=[[t1.y]], aggr=[[sum(alias1), max(alias1)]] 03)----Aggregate: groupBy=[[t1.y, __common_expr_1 AS t1.x AS alias1]], aggr=[[]] 04)------Projection: CAST(t1.x AS Float64) AS __common_expr_1, t1.y 05)--------TableScan: t1 projection=[x, y] physical_plan -01)ProjectionExec: expr=[sum(alias1)@1 as sum(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] -02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(alias1), MAX(alias1)] +01)ProjectionExec: expr=[sum(alias1)@1 as sum(DISTINCT t1.x), max(alias1)@2 as max(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), MAX(alias1)] +05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] 06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 @@ -4396,18 +4396,18 @@ EXPLAIN SELECT c1, count(distinct c2), min(distinct c2), sum(c3), max(c4) FROM a ---- logical_plan 01)Sort: aggregate_test_100.c1 ASC NULLS LAST -02)--Projection: aggregate_test_100.c1, count(alias1) AS count(DISTINCT aggregate_test_100.c2), MIN(alias1) AS MIN(DISTINCT aggregate_test_100.c2), sum(alias2) AS sum(aggregate_test_100.c3), MAX(alias3) AS MAX(aggregate_test_100.c4) -03)----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[count(alias1), MIN(alias1), sum(alias2), MAX(alias3)]] -04)------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[sum(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, MAX(aggregate_test_100.c4) AS alias3]] +02)--Projection: aggregate_test_100.c1, count(alias1) AS count(DISTINCT aggregate_test_100.c2), min(alias1) AS min(DISTINCT aggregate_test_100.c2), sum(alias2) AS sum(aggregate_test_100.c3), max(alias3) AS max(aggregate_test_100.c4) +03)----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[count(alias1), min(alias1), sum(alias2), max(alias3)]] +04)------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[sum(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, max(aggregate_test_100.c4) AS alias3]] 05)--------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4] physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[c1@0 as c1, count(alias1)@1 as count(DISTINCT aggregate_test_100.c2), MIN(alias1)@2 as MIN(DISTINCT aggregate_test_100.c2), sum(alias2)@3 as sum(aggregate_test_100.c3), MAX(alias3)@4 as MAX(aggregate_test_100.c4)] -04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[count(alias1), MIN(alias1), sum(alias2), MAX(alias3)] +03)----ProjectionExec: expr=[c1@0 as c1, count(alias1)@1 as count(DISTINCT aggregate_test_100.c2), min(alias1)@2 as min(DISTINCT aggregate_test_100.c2), sum(alias2)@3 as sum(aggregate_test_100.c3), max(alias3)@4 as max(aggregate_test_100.c4)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), MIN(alias1), sum(alias2), MAX(alias3)] +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[count(alias1), min(alias1), sum(alias2), max(alias3)] 08)--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 @@ -4576,17 +4576,17 @@ LIMIT 4; ---- logical_plan 01)Limit: skip=0, fetch=4 -02)--Sort: MAX(timestamp_table.t1) DESC NULLS FIRST, fetch=4 -03)----Aggregate: groupBy=[[timestamp_table.c2]], aggr=[[MAX(timestamp_table.t1)]] +02)--Sort: max(timestamp_table.t1) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[timestamp_table.c2]], aggr=[[max(timestamp_table.t1)]] 04)------TableScan: timestamp_table projection=[t1, c2] physical_plan 01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [MAX(timestamp_table.t1)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[MAX(timestamp_table.t1)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] +02)--SortPreservingMergeExec: [max(timestamp_table.t1)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(timestamp_table.t1)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[max(timestamp_table.t1)], lim=[4] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 -07)------------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] +07)------------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[max(timestamp_table.t1)], lim=[4] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 09)----------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 630237779bd7..087ad3ffbba0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -52,7 +52,7 @@ order by logical_plan 01)Sort: supplier.s_suppkey ASC NULLS LAST 02)--Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue -03)----Inner Join: revenue0.total_revenue = __scalar_sq_1.MAX(revenue0.total_revenue) +03)----Inner Join: revenue0.total_revenue = __scalar_sq_1.max(revenue0.total_revenue) 04)------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue 05)--------Inner Join: supplier.s_suppkey = revenue0.supplier_no 06)----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] @@ -63,7 +63,7 @@ logical_plan 11)------------------Filter: lineitem.l_shipdate >= Date32("1996-01-01") AND lineitem.l_shipdate < Date32("1996-04-01") 12)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("1996-01-01"), lineitem.l_shipdate < Date32("1996-04-01")] 13)------SubqueryAlias: __scalar_sq_1 -14)--------Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] +14)--------Aggregate: groupBy=[[]], aggr=[[max(revenue0.total_revenue)]] 15)----------SubqueryAlias: revenue0 16)------------Projection: sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue 17)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[sum(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] @@ -74,7 +74,7 @@ physical_plan 01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] 02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, max(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 07)------------CoalesceBatchesExec: target_batch_size=8192 @@ -93,10 +93,10 @@ physical_plan 20)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false 22)--------CoalesceBatchesExec: target_batch_size=8192 -23)----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 -24)------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +23)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 +24)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] 25)--------------CoalescePartitionsExec -26)----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +26)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] 27)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] 28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 29)----------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 1b1293cd250e..85dfefcd03f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -66,7 +66,7 @@ logical_plan 01)Limit: skip=0, fetch=10 02)--Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 03)----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment -04)------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) +04)------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.min(partsupp.ps_supplycost) 05)--------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name 06)----------Inner Join: nation.n_regionkey = region.r_regionkey 07)------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey @@ -85,8 +85,8 @@ logical_plan 20)--------------Filter: region.r_name = Utf8("EUROPE") 21)----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] 22)--------SubqueryAlias: __scalar_sq_1 -23)----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey -24)------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] +23)----------Projection: min(partsupp.ps_supplycost), partsupp.ps_partkey +24)------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[min(partsupp.ps_supplycost)]] 25)--------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost 26)----------------Inner Join: nation.n_regionkey = region.r_regionkey 27)------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey @@ -105,7 +105,7 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 @@ -149,12 +149,12 @@ physical_plan 47)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 48)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false 49)------------CoalesceBatchesExec: target_batch_size=8192 -50)--------------RepartitionExec: partitioning=Hash([ps_partkey@1, MIN(partsupp.ps_supplycost)@0], 4), input_partitions=4 -51)----------------ProjectionExec: expr=[MIN(partsupp.ps_supplycost)@1 as MIN(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -52)------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +50)--------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +51)----------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +52)------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 53)--------------------CoalesceBatchesExec: target_batch_size=8192 54)----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -55)------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +55)------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 56)--------------------------CoalesceBatchesExec: target_batch_size=8192 57)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] 58)------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 126996e7398a..aedbee35400c 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -588,8 +588,8 @@ logical_plan 06)----------EmptyRelation 07)--Projection: b.x AS count, b.y AS n 08)----SubqueryAlias: b -09)------Projection: Int64(1) AS x, MAX(Int64(10)) AS y -10)--------Aggregate: groupBy=[[]], aggr=[[MAX(Int64(10))]] +09)------Projection: Int64(1) AS x, max(Int64(10)) AS y +10)--------Aggregate: groupBy=[[]], aggr=[[max(Int64(10))]] 11)----------EmptyRelation physical_plan 01)UnionExec @@ -600,8 +600,8 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[count(*)], ordering_mode=Sorted 07)------------ProjectionExec: expr=[5 as n] 08)--------------PlaceholderRowExec -09)--ProjectionExec: expr=[1 as count, MAX(Int64(10))@0 as n] -10)----AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] +09)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] +10)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] 11)------PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index 3d455d7a88ca..59133379d443 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -53,8 +53,8 @@ logical_plan 01)Dml: op=[Update] table=[t1] 02)--Projection: t1.a AS a, () AS b, t1.c AS c, t1.d AS d 03)----Subquery: -04)------Projection: MAX(t2.b) -05)--------Aggregate: groupBy=[[]], aggr=[[MAX(t2.b)]] +04)------Projection: max(t2.b) +05)--------Aggregate: groupBy=[[]], aggr=[[max(t2.b)]] 06)----------Filter: outer_ref(t1.a) = t2.a 07)------------TableScan: t2 08)----TableScan: t1 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 212daa05a5d2..4f4b9749c561 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -254,8 +254,8 @@ WITH _sample_data AS ( ---- logical_plan 01)Sort: d.b ASC NULLS LAST -02)--Projection: d.b, MAX(d.a) AS max_a -03)----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a)]] +02)--Projection: d.b, max(d.a) AS max_a +03)----Aggregate: groupBy=[[d.b]], aggr=[[max(d.a)]] 04)------SubqueryAlias: d 05)--------SubqueryAlias: _data2 06)----------SubqueryAlias: s @@ -272,11 +272,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a] -04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] +03)----ProjectionExec: expr=[b@0 as b, max(d.a)@1 as max_a] +04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[max(d.a)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)], ordering_mode=Sorted +07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[max(d.a)], ordering_mode=Sorted 08)--------------UnionExec 09)----------------ProjectionExec: expr=[1 as a, aa as b] 10)------------------PlaceholderRowExec @@ -337,8 +337,8 @@ WITH _sample_data AS ( ---- logical_plan 01)Sort: d.b ASC NULLS LAST -02)--Projection: d.b, MAX(d.a) AS max_a, MAX(d.seq) -03)----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a), MAX(d.seq)]] +02)--Projection: d.b, max(d.a) AS max_a, max(d.seq) +03)----Aggregate: groupBy=[[d.b]], aggr=[[max(d.a), max(d.seq)]] 04)------SubqueryAlias: d 05)--------SubqueryAlias: _data2 06)----------Projection: ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS seq, s.a, s.b @@ -356,8 +356,8 @@ logical_plan 18)----------------------EmptyRelation physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] -02)--ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] -03)----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted +02)--ProjectionExec: expr=[b@0 as b, max(d.a)@1 as max_a, max(d.seq)@2 as max(d.seq)] +03)----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[max(d.a), max(d.seq)], ordering_mode=Sorted 04)------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] 05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 06)----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -1254,16 +1254,16 @@ query TT EXPLAIN SELECT c2, MAX(c9) OVER (ORDER BY c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100 ---- logical_plan -01)Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +01)Projection: aggregate_test_100.c2, max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW 02)--WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -03)----WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -04)------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----WindowAggr: windowExpr=[[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 05)--------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan -01)ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +01)ProjectionExec: expr=[c2@0 as c2, max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--WindowAggExec: wdw=[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -03)----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -04)------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true @@ -1277,18 +1277,18 @@ EXPLAIN SELECT c2, MAX(c9) OVER (ORDER BY c9, c2), SUM(c9) OVER (), MIN(c9) OVER ---- logical_plan 01)Sort: aggregate_test_100.c2 ASC NULLS LAST -02)--Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--Projection: aggregate_test_100.c2, max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW 03)----WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -04)------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -05)--------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 06)----------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan 01)SortExec: expr=[c2@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--ProjectionExec: expr=[c2@0 as c2, max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 03)----WindowAggExec: wdw=[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -04)------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST], preserve_partitioning=[false] -06)----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true @@ -2569,21 +2569,21 @@ logical_plan 01)Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, sumr1, sumr2, sumr3, minr1, minr2, minr3, maxr1, maxr2, maxr3, cntr1, cntr2, sum4, cnt3 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_finite.inc_col DESC NULLS FIRST, fetch=5 -04)------Projection: sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS sum2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS max3, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS cnt1, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING AS sumr1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING AS sumr2, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS maxr3, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS cntr1, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cntr2, sum(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS sum4, count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt3, annotated_data_finite.inc_col +04)------Projection: sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS sum2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS sum3, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS min1, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS min2, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS min3, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS max1, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS max2, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS max3, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS cnt1, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING AS sumr1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING AS sumr2, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sumr3, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS minr1, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS minr2, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS minr3, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS maxr1, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS maxr2, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS maxr3, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS cntr1, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cntr2, sum(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS sum4, count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt3, annotated_data_finite.inc_col 05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, count(Int64(1)) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] -06)----------Projection: __common_expr_1, annotated_data_finite.inc_col, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING -07)------------WindowAggr: windowExpr=[[sum(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] -08)--------------WindowAggr: windowExpr=[[sum(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +06)----------Projection: __common_expr_1, annotated_data_finite.inc_col, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING +07)------------WindowAggr: windowExpr=[[sum(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[sum(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, count(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] 09)----------------Projection: CAST(annotated_data_finite.desc_col AS Int64) AS __common_expr_1, CAST(annotated_data_finite.inc_col AS Int64) AS __common_expr_2, annotated_data_finite.ts, annotated_data_finite.inc_col, annotated_data_finite.desc_col 10)------------------TableScan: annotated_data_finite projection=[ts, inc_col, desc_col] physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3] 02)--SortExec: TopK(fetch=5), expr=[inc_col@24 DESC], preserve_partitioning=[false] -03)----ProjectionExec: expr=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as sum1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@14 as sum2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@15 as sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as max3, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@22 as cnt1, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@23 as cnt2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@2 as sumr1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@3 as sumr2, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@6 as minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as maxr3, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@11 as cntr1, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@12 as cntr2, sum(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as sum4, count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as cnt3, inc_col@1 as inc_col] +03)----ProjectionExec: expr=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as sum1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@14 as sum2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@15 as sum3, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as min1, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as min2, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as min3, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as max1, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as max2, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as max3, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@22 as cnt1, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@23 as cnt2, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@2 as sumr1, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@3 as sumr2, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sumr3, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as minr1, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@6 as minr2, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as minr3, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as maxr1, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as maxr2, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as maxr3, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@11 as cntr1, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@12 as cntr2, sum(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as sum4, count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as cnt3, inc_col@1 as inc_col] 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, inc_col@3 as inc_col, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] -06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] +05)--------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, inc_col@3 as inc_col, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] +06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] 08)--------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Int64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true @@ -2731,17 +2731,17 @@ logical_plan 01)Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -04)------Projection: sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS sum1, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS max2, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS count1, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS avg1, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS avg2, annotated_data_finite.inc_col -05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, avg(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, avg(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +04)------Projection: sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS sum1, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS min1, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS min2, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS max1, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS max2, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS count1, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS avg1, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS avg2, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, avg(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, avg(__common_expr_2 AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] 07)------------Projection: CAST(annotated_data_finite.inc_col AS Int64) AS __common_expr_1, CAST(annotated_data_finite.inc_col AS Float64) AS __common_expr_2, annotated_data_finite.ts, annotated_data_finite.inc_col 08)--------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2] 02)--SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST], preserve_partitioning=[false] -03)----ProjectionExec: expr=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as sum1, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as max2, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@12 as count1, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@7 as count2, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@13 as avg1, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@8 as avg2, inc_col@3 as inc_col] -04)------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] -05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as sum1, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as sum2, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as min1, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as min2, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as max1, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as max2, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@12 as count1, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@7 as count2, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@13 as avg1, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@8 as avg2, inc_col@3 as inc_col] +04)------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] 06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Float64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col] 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true @@ -3333,18 +3333,18 @@ EXPLAIN SELECT logical_plan 01)Limit: skip=0, fetch=5 02)--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -03)----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 -04)------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -05)--------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING -06)----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----Projection: aggregate_test_100.c3, max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 +04)------WindowAggr: windowExpr=[[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------Projection: aggregate_test_100.c3, aggregate_test_100.c12, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +06)----------WindowAggr: windowExpr=[[min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 07)------------TableScan: aggregate_test_100 projection=[c3, c11, c12] physical_plan 01)SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] -03)----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +02)--ProjectionExec: expr=[c3@0 as c3, max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] +03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c12@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -06)----------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +05)--------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +06)----------WindowAggExec: wdw=[min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 07)------------SortExec: expr=[c11@1 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true @@ -3379,14 +3379,14 @@ logical_plan 01)Projection: min1, max1 02)--Limit: skip=0, fetch=5 03)----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -04)------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1, aggregate_test_100.c3 -05)--------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------Projection: max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1, aggregate_test_100.c3 +05)--------WindowAggr: windowExpr=[[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 06)----------TableScan: aggregate_test_100 projection=[c3, c12] physical_plan 01)ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] 02)--SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST], preserve_partitioning=[false] -03)----ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] -04)------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] +04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c12@1 ASC NULLS LAST], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true @@ -3486,16 +3486,16 @@ EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, FROM multiple_ordered_table ---- logical_plan -01)Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 -02)--WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -03)----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -04)------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +01)Projection: min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 +02)--WindowAggr: windowExpr=[[min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: multiple_ordered_table.c, multiple_ordered_table.d, max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 05)--------TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan -01)ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] -02)--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -03)----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -04)------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +01)ProjectionExec: expr=[min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] +02)--BoundedWindowAggExec: wdw=[min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[c@2 as c, d@3 as d, max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query TT @@ -3506,13 +3506,13 @@ FROM( WHERE d=0) ---- logical_plan -01)Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c -02)--WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +01)Projection: max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c +02)--WindowAggr: windowExpr=[[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 03)----Filter: multiple_ordered_table.d = Int32(0) 04)------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] physical_plan -01)ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] -02)--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +01)ProjectionExec: expr=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] +02)--BoundedWindowAggExec: wdw=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------FilterExec: d@1 = 0 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true diff --git a/datafusion/substrait/tests/cases/consumer_integration.rs b/datafusion/substrait/tests/cases/consumer_integration.rs index fc5f82127d05..360377c231a3 100644 --- a/datafusion/substrait/tests/cases/consumer_integration.rs +++ b/datafusion/substrait/tests/cases/consumer_integration.rs @@ -98,7 +98,7 @@ mod tests { \n Projection: FILENAME_PLACEHOLDER_1.s_acctbal, FILENAME_PLACEHOLDER_1.s_name, FILENAME_PLACEHOLDER_3.n_name, FILENAME_PLACEHOLDER_0.p_partkey, FILENAME_PLACEHOLDER_0.p_mfgr, FILENAME_PLACEHOLDER_1.s_address, FILENAME_PLACEHOLDER_1.s_phone, FILENAME_PLACEHOLDER_1.s_comment\ \n Filter: FILENAME_PLACEHOLDER_0.p_partkey = FILENAME_PLACEHOLDER_2.ps_partkey AND FILENAME_PLACEHOLDER_1.s_suppkey = FILENAME_PLACEHOLDER_2.ps_suppkey AND FILENAME_PLACEHOLDER_0.p_size = Int32(15) AND FILENAME_PLACEHOLDER_0.p_type LIKE CAST(Utf8(\"%BRASS\") AS Utf8) AND FILENAME_PLACEHOLDER_1.s_nationkey = FILENAME_PLACEHOLDER_3.n_nationkey AND FILENAME_PLACEHOLDER_3.n_regionkey = FILENAME_PLACEHOLDER_4.r_regionkey AND FILENAME_PLACEHOLDER_4.r_name = CAST(Utf8(\"EUROPE\") AS Utf8) AND FILENAME_PLACEHOLDER_2.ps_supplycost = ()\ \n Subquery:\ - \n Aggregate: groupBy=[[]], aggr=[[MIN(FILENAME_PLACEHOLDER_5.ps_supplycost)]]\ + \n Aggregate: groupBy=[[]], aggr=[[min(FILENAME_PLACEHOLDER_5.ps_supplycost)]]\ \n Projection: FILENAME_PLACEHOLDER_5.ps_supplycost\ \n Filter: FILENAME_PLACEHOLDER_5.ps_partkey = FILENAME_PLACEHOLDER_5.ps_partkey AND FILENAME_PLACEHOLDER_6.s_suppkey = FILENAME_PLACEHOLDER_5.ps_suppkey AND FILENAME_PLACEHOLDER_6.s_nationkey = FILENAME_PLACEHOLDER_7.n_nationkey AND FILENAME_PLACEHOLDER_7.n_regionkey = FILENAME_PLACEHOLDER_8.r_regionkey AND FILENAME_PLACEHOLDER_8.r_name = CAST(Utf8(\"EUROPE\") AS Utf8)\ \n Inner Join: Filter: Boolean(true)\ diff --git a/docs/source/library-user-guide/using-the-sql-api.md b/docs/source/library-user-guide/using-the-sql-api.md index 9c32004db435..f78cf16f4cb6 100644 --- a/docs/source/library-user-guide/using-the-sql-api.md +++ b/docs/source/library-user-guide/using-the-sql-api.md @@ -52,13 +52,13 @@ async fn main() -> Result<()> { // register the "example" table ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?; // create a plan to run a SQL query - let df = ctx.sql("SELECT a, MIN(b) FROM example WHERE a <= b GROUP BY a LIMIT 100").await?; + let df = ctx.sql("SELECT a, min(b) FROM example WHERE a <= b GROUP BY a LIMIT 100").await?; // execute the plan and collect the results as Vec let results: Vec = df.collect().await?; // Use the assert_batches_eq macro to compare the results with expected output datafusion::assert_batches_eq!(vec![ "+---+----------------+", - "| a | MIN(example.b) |", + "| a | min(example.b) |", "+---+----------------+", "| 1 | 2 |", "+---+----------------+", From 5c4254aeb310731fff3fb29c95571e3db2efb41e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 5 Aug 2024 22:50:40 +0800 Subject: [PATCH 06/34] Don't implement create_sliding_accumulator repeatedly (#11813) --- datafusion/expr/src/test/function_stub.rs | 7 ------- datafusion/functions-aggregate/src/average.rs | 7 ------- datafusion/functions-aggregate/src/bool_and_or.rs | 14 -------------- datafusion/functions-aggregate/src/regr.rs | 7 ------- 4 files changed, 35 deletions(-) diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index 22af84db3f86..b4f768085fcc 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -202,13 +202,6 @@ impl AggregateUDFImpl for Sum { unreachable!("stub should not have accumulate()") } - fn create_sliding_accumulator( - &self, - _args: AccumulatorArgs, - ) -> Result> { - unreachable!("stub should not have accumulate()") - } - fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::Identical } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 228bce1979a3..288e0b09f809 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -215,13 +215,6 @@ impl AggregateUDFImpl for Avg { &self.aliases } - fn create_sliding_accumulator( - &self, - args: AccumulatorArgs, - ) -> Result> { - self.accumulator(args) - } - fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::Identical } diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs index d0028672743e..b91fbb9ff709 100644 --- a/datafusion/functions-aggregate/src/bool_and_or.rs +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -165,13 +165,6 @@ impl AggregateUDFImpl for BoolAnd { &[] } - fn create_sliding_accumulator( - &self, - _: AccumulatorArgs, - ) -> Result> { - Ok(Box::::default()) - } - fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } @@ -292,13 +285,6 @@ impl AggregateUDFImpl for BoolOr { &[] } - fn create_sliding_accumulator( - &self, - _: AccumulatorArgs, - ) -> Result> { - Ok(Box::::default()) - } - fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } diff --git a/datafusion/functions-aggregate/src/regr.rs b/datafusion/functions-aggregate/src/regr.rs index aad110a13e13..390a769aca7f 100644 --- a/datafusion/functions-aggregate/src/regr.rs +++ b/datafusion/functions-aggregate/src/regr.rs @@ -164,13 +164,6 @@ impl AggregateUDFImpl for Regr { Ok(Box::new(RegrAccumulator::try_new(&self.regr_type)?)) } - fn create_sliding_accumulator( - &self, - _args: AccumulatorArgs, - ) -> Result> { - Ok(Box::new(RegrAccumulator::try_new(&self.regr_type)?)) - } - fn state_fields(&self, args: StateFieldsArgs) -> Result> { Ok(vec![ Field::new( From f19d30d5261e3a525e2898a8f08f44744820e605 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 11:07:46 -0400 Subject: [PATCH 07/34] chore(deps): update rstest requirement from 0.21.0 to 0.22.0 (#11811) Updates the requirements on [rstest](https://github.com/la10736/rstest) to permit the latest version. - [Release notes](https://github.com/la10736/rstest/releases) - [Changelog](https://github.com/la10736/rstest/blob/master/CHANGELOG.md) - [Commits](https://github.com/la10736/rstest/compare/v0.21.0...v0.21.0) --- updated-dependencies: - dependency-name: rstest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 90aff3f715ca..38b5627bc187 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -125,7 +125,7 @@ parquet = { version = "52.2.0", default-features = false, features = [ ] } rand = "0.8" regex = "1.8" -rstest = "0.21.0" +rstest = "0.22.0" serde_json = "1" sqlparser = { version = "0.49", features = ["visitor"] } tempfile = "3" From 0417e543e1b0de61f9eb48c9c1dd4af943faa394 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 5 Aug 2024 12:00:58 -0400 Subject: [PATCH 08/34] Minor: Update exected output due to logical conflict (#11824) --- datafusion/sqllogictest/test_files/aggregate.slt | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 9625f02afbb4..8a5222143356 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4530,21 +4530,21 @@ EXPLAIN SELECT DISTINCT c3, min(c1) FROM aggregate_test_100 group by c3 limit 5; ---- logical_plan 01)Limit: skip=0, fetch=5 -02)--Aggregate: groupBy=[[aggregate_test_100.c3, MIN(aggregate_test_100.c1)]], aggr=[[]] -03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[MIN(aggregate_test_100.c1)]] +02)--Aggregate: groupBy=[[aggregate_test_100.c3, min(aggregate_test_100.c1)]], aggr=[[]] +03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[min(aggregate_test_100.c1)]] 04)------TableScan: aggregate_test_100 projection=[c1, c3] physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--CoalescePartitionsExec 03)----LocalLimitExec: fetch=5 -04)------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, MIN(aggregate_test_100.c1)@1 as MIN(aggregate_test_100.c1)], aggr=[], lim=[5] +04)------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([c3@0, MIN(aggregate_test_100.c1)@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[c3@0 as c3, MIN(aggregate_test_100.c1)@1 as MIN(aggregate_test_100.c1)], aggr=[], lim=[5] -08)--------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[MIN(aggregate_test_100.c1)] +06)----------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 -11)--------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[MIN(aggregate_test_100.c1)] +11)--------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true From f56a2ef28f669351d5faec23ca0a6edef5c53ee6 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 6 Aug 2024 00:21:04 +0800 Subject: [PATCH 09/34] doc: Add support for `map` and `make_map` functions (#11799) * doc: Add support for `map` and `make_map` functions * chore: Add example for MAP --- .../source/user-guide/sql/scalar_functions.md | 64 +++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 561824772af8..c7490df04983 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3636,6 +3636,70 @@ Unwraps struct fields into columns. +-----------------------+-----------------------+ ``` +## Map Functions + +- [map](#map) +- [make_map](#make_map) + +### `map` + +Returns an Arrow map with the specified key-value pairs. + +``` +map(key, value) +map(key: value) +``` + +#### Arguments + +- **key**: Expression to be used for key. + Can be a constant, column, or function, any combination of arithmetic or + string operators, or a named expression of previous listed. +- **value**: Expression to be used for value. + Can be a constant, column, or function, any combination of arithmetic or + string operators, or a named expression of previous listed. + +#### Example + +``` +SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33, null]); +---- +{POST: 41, HEAD: 33, PATCH: } + +SELECT MAP([[1,2], [3,4]], ['a', 'b']); +---- +{[1, 2]: a, [3, 4]: b} + +SELECT MAP { 'a': 1, 'b': 2 }; +---- +{a: 1, b: 2} +``` + +### `make_map` + +Returns an Arrow map with the specified key-value pairs. + +``` +make_map(key_1, value_1, ..., key_n, value_n) +``` + +#### Arguments + +- **key_n**: Expression to be used for key. + Can be a constant, column, or function, any combination of arithmetic or + string operators, or a named expression of previous listed. +- **value_n**: Expression to be used for value. + Can be a constant, column, or function, any combination of arithmetic or + string operators, or a named expression of previous listed. + +#### Example + +``` +SELECT MAKE_MAP('POST', 41, 'HEAD', 33, 'PATCH', null); +---- +{POST: 41, HEAD: 33, PATCH: } +``` + ## Hashing Functions - [digest](#digest) From 011a3f3e32e140190486d51cd98494de29be9ba5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 6 Aug 2024 02:31:36 +0800 Subject: [PATCH 10/34] Improve readme page in crates.io (#11809) --- datafusion/core/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 0415c3164b38..0714c3e94a85 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -20,7 +20,7 @@ name = "datafusion" description = "DataFusion is an in-memory query engine that uses Apache Arrow as the memory model" keywords = ["arrow", "query", "sql"] include = ["benches/*.rs", "src/**/*.rs", "Cargo.toml"] -readme = "README.md" +readme = "../../README.md" version = { workspace = true } edition = { workspace = true } homepage = { workspace = true } From c6f0d3cac93ef1436313160f1dba878745d693bb Mon Sep 17 00:00:00 2001 From: Simon Vandel Sillesen Date: Mon, 5 Aug 2024 20:32:15 +0200 Subject: [PATCH 11/34] Pass scalar to `eq` inside `nullif` (#11697) * Properly specialize nullif for scalar (3x faster) * missed feature flag * fix test * extract * dodes -> does Co-authored-by: Oleks V --------- Co-authored-by: Oleks V --- datafusion/functions/Cargo.toml | 5 +++ datafusion/functions/benches/nullif.rs | 42 +++++++++++++++++++++++++ datafusion/functions/src/core/nullif.rs | 10 ++++-- 3 files changed, 54 insertions(+), 3 deletions(-) create mode 100644 datafusion/functions/benches/nullif.rs diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index 0281676cabf2..9675d03a0161 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -112,6 +112,11 @@ harness = false name = "make_date" required-features = ["datetime_expressions"] +[[bench]] +harness = false +name = "nullif" +required-features = ["core_expressions"] + [[bench]] harness = false name = "date_bin" diff --git a/datafusion/functions/benches/nullif.rs b/datafusion/functions/benches/nullif.rs new file mode 100644 index 000000000000..dfabad335835 --- /dev/null +++ b/datafusion/functions/benches/nullif.rs @@ -0,0 +1,42 @@ +// 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. + +extern crate criterion; + +use arrow::util::bench_util::create_string_array_with_len; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_common::ScalarValue; +use datafusion_expr::ColumnarValue; +use datafusion_functions::core::nullif; +use std::sync::Arc; + +fn criterion_benchmark(c: &mut Criterion) { + let nullif = nullif(); + for size in [1024, 4096, 8192] { + let array = Arc::new(create_string_array_with_len::(size, 0.2, 32)); + let args = vec![ + ColumnarValue::Scalar(ScalarValue::Utf8(Some("abcd".to_string()))), + ColumnarValue::Array(array), + ]; + c.bench_function(&format!("nullif scalar array: {}", size), |b| { + b.iter(|| black_box(nullif.invoke(&args).unwrap())) + }); + } +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions/src/core/nullif.rs b/datafusion/functions/src/core/nullif.rs index e8bf2db514c3..6fcfbd36416e 100644 --- a/datafusion/functions/src/core/nullif.rs +++ b/datafusion/functions/src/core/nullif.rs @@ -19,7 +19,6 @@ use arrow::datatypes::DataType; use datafusion_common::{exec_err, Result}; use datafusion_expr::ColumnarValue; -use arrow::array::Array; use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::nullif::nullif; use datafusion_common::ScalarValue; @@ -122,8 +121,13 @@ fn nullif_func(args: &[ColumnarValue]) -> Result { Ok(ColumnarValue::Array(array)) } (ColumnarValue::Scalar(lhs), ColumnarValue::Array(rhs)) => { - let lhs = lhs.to_array_of_size(rhs.len())?; - let array = nullif(&lhs, &eq(&lhs, &rhs)?)?; + let lhs_s = lhs.to_scalar()?; + let lhs_a = lhs.to_array_of_size(rhs.len())?; + let array = nullif( + // nullif in arrow-select does not support Datum, so we need to convert to array + lhs_a.as_ref(), + &eq(&lhs_s, &rhs)?, + )?; Ok(ColumnarValue::Array(array)) } (ColumnarValue::Scalar(lhs), ColumnarValue::Scalar(rhs)) => { From fcd907d78e32976b4f4dd93db5fb87a385630acd Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 6 Aug 2024 04:16:23 +0800 Subject: [PATCH 12/34] refactor: move `aggregate_statistics` to `datafusion-physical-optimizer` (#11798) * refactor: move aggregate statistics to datafusion-physical-optimizer * chore * chore: Update cargo lock * refactor: Move COUNT_STAR_EXPANSION to datafusion_common * refactor: Move tests to core/tests --- datafusion-cli/Cargo.lock | 100 +-- datafusion/common/src/utils/expr.rs | 24 + datafusion/common/src/utils/mod.rs | 1 + .../aggregate_statistics.rs | 657 ------------------ .../limited_distinct_aggregation.rs | 2 +- datafusion/core/src/physical_optimizer/mod.rs | 1 - datafusion/core/src/test_util/mod.rs | 61 +- .../tests/physical_optimizer_integration.rs | 325 +++++++++ datafusion/expr/src/utils.rs | 4 +- .../src/aggregate_statistics.rs | 298 ++++++++ datafusion/physical-optimizer/src/lib.rs | 1 + 11 files changed, 770 insertions(+), 704 deletions(-) create mode 100644 datafusion/common/src/utils/expr.rs delete mode 100644 datafusion/core/src/physical_optimizer/aggregate_statistics.rs create mode 100644 datafusion/core/tests/physical_optimizer_integration.rs create mode 100644 datafusion/physical-optimizer/src/aggregate_statistics.rs diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5884e424c781..9d20c242bbef 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -272,7 +272,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.2.6", + "indexmap 2.3.0", "lexical-core", "num", "serde", @@ -375,7 +375,7 @@ dependencies = [ "tokio", "xz2", "zstd 0.13.2", - "zstd-safe 7.2.0", + "zstd-safe 7.2.1", ] [[package]] @@ -837,9 +837,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.6.1" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12916984aab3fa6e39d655a33e09c0071eb36d6ab3aea5c2d78551f1df6d952" +checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" [[package]] name = "bytes-utils" @@ -874,9 +874,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.6" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2aba8f4e9906c7ce3c73463f62a7f0c65183ada1a2d47e397cc8810827f9694f" +checksum = "26a5c3fd7bfa1ce3897a3a3501d362b2d87b7f2583ebcb4a949ec25911025cbc" dependencies = [ "jobserver", "libc", @@ -1161,7 +1161,7 @@ dependencies = [ "glob", "half", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "num-traits", @@ -1357,7 +1357,7 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "paste", @@ -1384,7 +1384,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "hex", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "paste", @@ -1436,7 +1436,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.3.0", "itertools 0.12.1", "log", "once_cell", @@ -1629,9 +1629,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.30" +version = "1.0.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" +checksum = "7f211bbe8e69bbd0cfdea405084f128ae8b4aaa6b0b522fc8f2b009084797920" dependencies = [ "crc32fast", "miniz_oxide", @@ -1801,7 +1801,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.2.6", + "indexmap 2.3.0", "slab", "tokio", "tokio-util", @@ -1820,7 +1820,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.1.0", - "indexmap 2.2.6", + "indexmap 2.3.0", "slab", "tokio", "tokio-util", @@ -2112,9 +2112,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.6" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" +checksum = "de3fc2e30ba82dd1b3911c8de1ffc143c74a914a14e99514d7637e3099df5ea0" dependencies = [ "equivalent", "hashbrown 0.14.5", @@ -2552,7 +2552,7 @@ dependencies = [ "rand", "reqwest", "ring 0.17.8", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "serde", "serde_json", "snafu", @@ -2682,7 +2682,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.2.6", + "indexmap 2.3.0", ] [[package]] @@ -2769,9 +2769,12 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] [[package]] name = "predicates" @@ -2854,9 +2857,9 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4ceeeeabace7857413798eb1ffa1e9c905a9946a57d81fb69b4b71c4d8eb3ad" +checksum = "b22d8e7369034b9a7132bc2008cac12f2013c8132b45e0554e6e20e2617f2156" dependencies = [ "bytes", "pin-project-lite", @@ -2864,6 +2867,7 @@ dependencies = [ "quinn-udp", "rustc-hash", "rustls 0.23.12", + "socket2", "thiserror", "tokio", "tracing", @@ -2871,9 +2875,9 @@ dependencies = [ [[package]] name = "quinn-proto" -version = "0.11.3" +version = "0.11.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddf517c03a109db8100448a4be38d498df8a210a99fe0e1b9eaf39e78c640efe" +checksum = "ba92fb39ec7ad06ca2582c0ca834dfeadcaf06ddfc8e635c80aa7e1c05315fdd" dependencies = [ "bytes", "rand", @@ -2895,6 +2899,7 @@ dependencies = [ "libc", "once_cell", "socket2", + "tracing", "windows-sys 0.52.0", ] @@ -2969,9 +2974,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.5" +version = "1.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f" +checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" dependencies = [ "aho-corasick", "memchr", @@ -3029,7 +3034,7 @@ dependencies = [ "quinn", "rustls 0.23.12", "rustls-native-certs 0.7.1", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", "serde_json", @@ -3117,9 +3122,9 @@ checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" [[package]] name = "rustc-hash" -version = "1.1.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" +checksum = "583034fd73374156e66797ed8e5b0d5690409c9226b22d87cb7f19821c05d152" [[package]] name = "rustc_version" @@ -3188,7 +3193,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a88d6d420651b496bdd98684116959239430022a115c1240e6c3993be0b15fba" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "rustls-pki-types", "schannel", "security-framework", @@ -3205,9 +3210,9 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.2" +version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" +checksum = "196fe16b00e106300d3e45ecfcb764fa292a535d7326a29a5875c579c7417425" dependencies = [ "base64 0.22.1", "rustls-pki-types", @@ -3356,9 +3361,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.121" +version = "1.0.122" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ab380d7d9f22ef3f21ad3e6c1ebe8e4fc7a2000ccba2e4d71fc96f15b2cb609" +checksum = "784b6203951c57ff748476b126ccb5e8e2959a5c19e5c617ab1956be3dbc68da" dependencies = [ "itoa", "memchr", @@ -3585,12 +3590,13 @@ checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" [[package]] name = "tempfile" -version = "3.10.1" +version = "3.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +checksum = "b8fcd239983515c23a32fb82099f97d0b11b8c72f654ed659363a95c3dad7a53" dependencies = [ "cfg-if", "fastrand 2.1.0", + "once_cell", "rustix", "windows-sys 0.52.0", ] @@ -4119,11 +4125,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -4159,6 +4165,15 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", +] + [[package]] name = "windows-targets" version = "0.48.5" @@ -4311,6 +4326,7 @@ version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ + "byteorder", "zerocopy-derive", ] @@ -4346,7 +4362,7 @@ version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" dependencies = [ - "zstd-safe 7.2.0", + "zstd-safe 7.2.1", ] [[package]] @@ -4361,9 +4377,9 @@ dependencies = [ [[package]] name = "zstd-safe" -version = "7.2.0" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa556e971e7b568dc775c136fc9de8c779b1c2fc3a63defaafadffdbd3181afa" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" dependencies = [ "zstd-sys", ] diff --git a/datafusion/common/src/utils/expr.rs b/datafusion/common/src/utils/expr.rs new file mode 100644 index 000000000000..0fe4546b8538 --- /dev/null +++ b/datafusion/common/src/utils/expr.rs @@ -0,0 +1,24 @@ +// 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. + +//! Expression utilities + +use crate::ScalarValue; + +/// The value to which `COUNT(*)` is expanded to in +/// `COUNT()` expressions +pub const COUNT_STAR_EXPANSION: ScalarValue = ScalarValue::Int64(Some(1)); diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 8b025255f5df..58dc8f40b577 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -17,6 +17,7 @@ //! This module provides the bisect function, which implements binary search. +pub mod expr; pub mod memory; pub mod proxy; diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs deleted file mode 100644 index a0f6f6a65b1f..000000000000 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ /dev/null @@ -1,657 +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. - -//! Utilizing exact statistics from sources to avoid scanning data -use std::sync::Arc; - -use crate::config::ConfigOptions; -use crate::error::Result; -use crate::physical_plan::aggregates::AggregateExec; -use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; -use crate::scalar::ScalarValue; - -use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_expr::utils::COUNT_STAR_EXPANSION; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; -use datafusion_physical_plan::udaf::AggregateFunctionExpr; - -/// Optimizer that uses available statistics for aggregate functions -#[derive(Default)] -pub struct AggregateStatistics {} - -impl AggregateStatistics { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for AggregateStatistics { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - if let Some(partial_agg_exec) = take_optimizable(&*plan) { - let partial_agg_exec = partial_agg_exec - .as_any() - .downcast_ref::() - .expect("take_optimizable() ensures that this is a AggregateExec"); - let stats = partial_agg_exec.input().statistics()?; - let mut projections = vec![]; - for expr in partial_agg_exec.aggr_expr() { - if let Some((non_null_rows, name)) = - take_optimizable_column_and_table_count(&**expr, &stats) - { - projections.push((expressions::lit(non_null_rows), name.to_owned())); - } else if let Some((min, name)) = take_optimizable_min(&**expr, &stats) { - projections.push((expressions::lit(min), name.to_owned())); - } else if let Some((max, name)) = take_optimizable_max(&**expr, &stats) { - projections.push((expressions::lit(max), name.to_owned())); - } else { - // TODO: we need all aggr_expr to be resolved (cf TODO fullres) - break; - } - } - - // TODO fullres: use statistics even if not all aggr_expr could be resolved - if projections.len() == partial_agg_exec.aggr_expr().len() { - // input can be entirely removed - Ok(Arc::new(ProjectionExec::try_new( - projections, - Arc::new(PlaceholderRowExec::new(plan.schema())), - )?)) - } else { - plan.map_children(|child| { - self.optimize(child, _config).map(Transformed::yes) - }) - .data() - } - } else { - plan.map_children(|child| self.optimize(child, _config).map(Transformed::yes)) - .data() - } - } - - fn name(&self) -> &str { - "aggregate_statistics" - } - - /// This rule will change the nullable properties of the schema, disable the schema check. - fn schema_check(&self) -> bool { - false - } -} - -/// assert if the node passed as argument is a final `AggregateExec` node that can be optimized: -/// - its child (with possible intermediate layers) is a partial `AggregateExec` node -/// - they both have no grouping expression -/// -/// If this is the case, return a ref to the partial `AggregateExec`, else `None`. -/// We would have preferred to return a casted ref to AggregateExec but the recursion requires -/// the `ExecutionPlan.children()` method that returns an owned reference. -fn take_optimizable(node: &dyn ExecutionPlan) -> Option> { - if let Some(final_agg_exec) = node.as_any().downcast_ref::() { - if !final_agg_exec.mode().is_first_stage() - && final_agg_exec.group_expr().is_empty() - { - let mut child = Arc::clone(final_agg_exec.input()); - loop { - if let Some(partial_agg_exec) = - child.as_any().downcast_ref::() - { - if partial_agg_exec.mode().is_first_stage() - && partial_agg_exec.group_expr().is_empty() - && partial_agg_exec.filter_expr().iter().all(|e| e.is_none()) - { - return Some(child); - } - } - if let [childrens_child] = child.children().as_slice() { - child = Arc::clone(childrens_child); - } else { - break; - } - } - } - } - None -} - -/// If this agg_expr is a count that can be exactly derived from the statistics, return it. -fn take_optimizable_column_and_table_count( - agg_expr: &dyn AggregateExpr, - stats: &Statistics, -) -> Option<(ScalarValue, String)> { - let col_stats = &stats.column_statistics; - if is_non_distinct_count(agg_expr) { - if let Precision::Exact(num_rows) = stats.num_rows { - let exprs = agg_expr.expressions(); - if exprs.len() == 1 { - // TODO optimize with exprs other than Column - if let Some(col_expr) = - exprs[0].as_any().downcast_ref::() - { - let current_val = &col_stats[col_expr.index()].null_count; - if let &Precision::Exact(val) = current_val { - return Some(( - ScalarValue::Int64(Some((num_rows - val) as i64)), - agg_expr.name().to_string(), - )); - } - } else if let Some(lit_expr) = - exprs[0].as_any().downcast_ref::() - { - if lit_expr.value() == &COUNT_STAR_EXPANSION { - return Some(( - ScalarValue::Int64(Some(num_rows as i64)), - agg_expr.name().to_string(), - )); - } - } - } - } - } - None -} - -/// If this agg_expr is a min that is exactly defined in the statistics, return it. -fn take_optimizable_min( - agg_expr: &dyn AggregateExpr, - stats: &Statistics, -) -> Option<(ScalarValue, String)> { - if let Precision::Exact(num_rows) = &stats.num_rows { - match *num_rows { - 0 => { - // MIN/MAX with 0 rows is always null - if is_min(agg_expr) { - if let Ok(min_data_type) = - ScalarValue::try_from(agg_expr.field().unwrap().data_type()) - { - return Some((min_data_type, agg_expr.name().to_string())); - } - } - } - value if value > 0 => { - let col_stats = &stats.column_statistics; - if is_min(agg_expr) { - let exprs = agg_expr.expressions(); - if exprs.len() == 1 { - // TODO optimize with exprs other than Column - if let Some(col_expr) = - exprs[0].as_any().downcast_ref::() - { - if let Precision::Exact(val) = - &col_stats[col_expr.index()].min_value - { - if !val.is_null() { - return Some(( - val.clone(), - agg_expr.name().to_string(), - )); - } - } - } - } - } - } - _ => {} - } - } - None -} - -/// If this agg_expr is a max that is exactly defined in the statistics, return it. -fn take_optimizable_max( - agg_expr: &dyn AggregateExpr, - stats: &Statistics, -) -> Option<(ScalarValue, String)> { - if let Precision::Exact(num_rows) = &stats.num_rows { - match *num_rows { - 0 => { - // MIN/MAX with 0 rows is always null - if is_max(agg_expr) { - if let Ok(max_data_type) = - ScalarValue::try_from(agg_expr.field().unwrap().data_type()) - { - return Some((max_data_type, agg_expr.name().to_string())); - } - } - } - value if value > 0 => { - let col_stats = &stats.column_statistics; - if is_max(agg_expr) { - let exprs = agg_expr.expressions(); - if exprs.len() == 1 { - // TODO optimize with exprs other than Column - if let Some(col_expr) = - exprs[0].as_any().downcast_ref::() - { - if let Precision::Exact(val) = - &col_stats[col_expr.index()].max_value - { - if !val.is_null() { - return Some(( - val.clone(), - agg_expr.name().to_string(), - )); - } - } - } - } - } - } - _ => {} - } - } - None -} - -// TODO: Move this check into AggregateUDFImpl -// https://github.com/apache/datafusion/issues/11153 -fn is_non_distinct_count(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { - return true; - } - } - false -} - -// TODO: Move this check into AggregateUDFImpl -// https://github.com/apache/datafusion/issues/11153 -fn is_min(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name().to_lowercase() == "min" { - return true; - } - } - false -} - -// TODO: Move this check into AggregateUDFImpl -// https://github.com/apache/datafusion/issues/11153 -fn is_max(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name().to_lowercase() == "max" { - return true; - } - } - false -} - -#[cfg(test)] -pub(crate) mod tests { - use super::*; - - use crate::logical_expr::Operator; - use crate::physical_plan::aggregates::PhysicalGroupBy; - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::common; - use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::memory::MemoryExec; - use crate::prelude::SessionContext; - - use arrow::array::Int32Array; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_common::cast::as_int64_array; - use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::expressions::cast; - use datafusion_physical_expr::PhysicalExpr; - use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; - use datafusion_physical_plan::aggregates::AggregateMode; - - /// Mock data using a MemoryExec which has an exact count statistic - fn mock_data() -> Result> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ])); - - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![Some(1), Some(2), None])), - Arc::new(Int32Array::from(vec![Some(4), None, Some(6)])), - ], - )?; - - Ok(Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?)) - } - - /// Checks that the count optimization was applied and we still get the right result - async fn assert_count_optim_success( - plan: AggregateExec, - agg: TestAggregate, - ) -> Result<()> { - let session_ctx = SessionContext::new(); - let state = session_ctx.state(); - let plan: Arc = Arc::new(plan); - - let optimized = AggregateStatistics::new() - .optimize(Arc::clone(&plan), state.config_options())?; - - // A ProjectionExec is a sign that the count optimization was applied - assert!(optimized.as_any().is::()); - - // run both the optimized and nonoptimized plan - let optimized_result = - common::collect(optimized.execute(0, session_ctx.task_ctx())?).await?; - let nonoptimized_result = - common::collect(plan.execute(0, session_ctx.task_ctx())?).await?; - assert_eq!(optimized_result.len(), nonoptimized_result.len()); - - // and validate the results are the same and expected - assert_eq!(optimized_result.len(), 1); - check_batch(optimized_result.into_iter().next().unwrap(), &agg); - // check the non optimized one too to ensure types and names remain the same - assert_eq!(nonoptimized_result.len(), 1); - check_batch(nonoptimized_result.into_iter().next().unwrap(), &agg); - - Ok(()) - } - - fn check_batch(batch: RecordBatch, agg: &TestAggregate) { - let schema = batch.schema(); - let fields = schema.fields(); - assert_eq!(fields.len(), 1); - - let field = &fields[0]; - assert_eq!(field.name(), agg.column_name()); - assert_eq!(field.data_type(), &DataType::Int64); - // note that nullabiolity differs - - assert_eq!( - as_int64_array(batch.column(0)).unwrap().values(), - &[agg.expected_count()] - ); - } - - /// Describe the type of aggregate being tested - pub(crate) enum TestAggregate { - /// Testing COUNT(*) type aggregates - CountStar, - - /// Testing for COUNT(column) aggregate - ColumnA(Arc), - } - - impl TestAggregate { - pub(crate) fn new_count_star() -> Self { - Self::CountStar - } - - fn new_count_column(schema: &Arc) -> Self { - Self::ColumnA(schema.clone()) - } - - // Return appropriate expr depending if COUNT is for col or table (*) - pub(crate) fn count_expr(&self, schema: &Schema) -> Arc { - AggregateExprBuilder::new(count_udaf(), vec![self.column()]) - .schema(Arc::new(schema.clone())) - .name(self.column_name()) - .build() - .unwrap() - } - - /// what argument would this aggregate need in the plan? - fn column(&self) -> Arc { - match self { - Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), - Self::ColumnA(s) => expressions::col("a", s).unwrap(), - } - } - - /// What name would this aggregate produce in a plan? - fn column_name(&self) -> &'static str { - match self { - Self::CountStar => "COUNT(*)", - Self::ColumnA(_) => "COUNT(a)", - } - } - - /// What is the expected count? - fn expected_count(&self) -> i64 { - match self { - TestAggregate::CountStar => 3, - TestAggregate::ColumnA(_) => 2, - } - } - } - - #[tokio::test] - async fn test_count_partial_direct_child() -> Result<()> { - // basic test case with the aggregation applied on a source with exact statistics - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_star(); - - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - source, - Arc::clone(&schema), - )?; - - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - Arc::new(partial_agg), - Arc::clone(&schema), - )?; - - assert_count_optim_success(final_agg, agg).await?; - - Ok(()) - } - - #[tokio::test] - async fn test_count_partial_with_nulls_direct_child() -> Result<()> { - // basic test case with the aggregation applied on a source with exact statistics - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_column(&schema); - - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - source, - Arc::clone(&schema), - )?; - - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - Arc::new(partial_agg), - Arc::clone(&schema), - )?; - - assert_count_optim_success(final_agg, agg).await?; - - Ok(()) - } - - #[tokio::test] - async fn test_count_partial_indirect_child() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_star(); - - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - source, - Arc::clone(&schema), - )?; - - // We introduce an intermediate optimization step between the partial and final aggregtator - let coalesce = CoalescePartitionsExec::new(Arc::new(partial_agg)); - - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - Arc::new(coalesce), - Arc::clone(&schema), - )?; - - assert_count_optim_success(final_agg, agg).await?; - - Ok(()) - } - - #[tokio::test] - async fn test_count_partial_with_nulls_indirect_child() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_column(&schema); - - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - source, - Arc::clone(&schema), - )?; - - // We introduce an intermediate optimization step between the partial and final aggregtator - let coalesce = CoalescePartitionsExec::new(Arc::new(partial_agg)); - - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - Arc::new(coalesce), - Arc::clone(&schema), - )?; - - assert_count_optim_success(final_agg, agg).await?; - - Ok(()) - } - - #[tokio::test] - async fn test_count_inexact_stat() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_star(); - - // adding a filter makes the statistics inexact - let filter = Arc::new(FilterExec::try_new( - expressions::binary( - expressions::col("a", &schema)?, - Operator::Gt, - cast(expressions::lit(1u32), &schema, DataType::Int32)?, - &schema, - )?, - source, - )?); - - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - filter, - Arc::clone(&schema), - )?; - - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - Arc::new(partial_agg), - Arc::clone(&schema), - )?; - - let conf = ConfigOptions::new(); - let optimized = - AggregateStatistics::new().optimize(Arc::new(final_agg), &conf)?; - - // check that the original ExecutionPlan was not replaced - assert!(optimized.as_any().is::()); - - Ok(()) - } - - #[tokio::test] - async fn test_count_with_nulls_inexact_stat() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_column(&schema); - - // adding a filter makes the statistics inexact - let filter = Arc::new(FilterExec::try_new( - expressions::binary( - expressions::col("a", &schema)?, - Operator::Gt, - cast(expressions::lit(1u32), &schema, DataType::Int32)?, - &schema, - )?, - source, - )?); - - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - filter, - Arc::clone(&schema), - )?; - - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - PhysicalGroupBy::default(), - vec![agg.count_expr(&schema)], - vec![None], - Arc::new(partial_agg), - Arc::clone(&schema), - )?; - - let conf = ConfigOptions::new(); - let optimized = - AggregateStatistics::new().optimize(Arc::new(final_agg), &conf)?; - - // check that the original ExecutionPlan was not replaced - assert!(optimized.as_any().is::()); - - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index b5d3f432d84d..b181ad9051ed 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -193,7 +193,6 @@ impl PhysicalOptimizerRule for LimitedDistinctAggregation { mod tests { use super::*; - use crate::physical_optimizer::aggregate_statistics::tests::TestAggregate; use crate::physical_optimizer::enforce_distribution::tests::{ parquet_exec_with_sort, schema, trim_plan_display, }; @@ -201,6 +200,7 @@ mod tests { use crate::physical_plan::collect; use crate::physical_plan::memory::MemoryExec; use crate::prelude::SessionContext; + use crate::test_util::TestAggregate; use arrow::array::Int32Array; use arrow::compute::SortOptions; diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 01ddab3ec97d..9291d0b84865 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -21,7 +21,6 @@ //! "Repartition" or "Sortedness" //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan -pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; pub mod enforce_distribution; diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 042febf32fd1..6eb82dece31c 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -45,11 +45,16 @@ use crate::prelude::{CsvReadOptions, SessionContext}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::TableReference; +use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::{CreateExternalTable, Expr, TableType}; -use datafusion_physical_expr::EquivalenceProperties; +use datafusion_functions_aggregate::count::count_udaf; +use datafusion_physical_expr::{ + expressions, AggregateExpr, EquivalenceProperties, PhysicalExpr, +}; use async_trait::async_trait; use datafusion_catalog::Session; +use datafusion_physical_expr_common::aggregate::AggregateExprBuilder; use futures::Stream; use tempfile::TempDir; // backwards compatibility @@ -402,3 +407,57 @@ pub fn bounded_stream(batch: RecordBatch, limit: usize) -> SendableRecordBatchSt batch, }) } + +/// Describe the type of aggregate being tested +pub enum TestAggregate { + /// Testing COUNT(*) type aggregates + CountStar, + + /// Testing for COUNT(column) aggregate + ColumnA(Arc), +} + +impl TestAggregate { + /// Create a new COUNT(*) aggregate + pub fn new_count_star() -> Self { + Self::CountStar + } + + /// Create a new COUNT(column) aggregate + pub fn new_count_column(schema: &Arc) -> Self { + Self::ColumnA(schema.clone()) + } + + /// Return appropriate expr depending if COUNT is for col or table (*) + pub fn count_expr(&self, schema: &Schema) -> Arc { + AggregateExprBuilder::new(count_udaf(), vec![self.column()]) + .schema(Arc::new(schema.clone())) + .name(self.column_name()) + .build() + .unwrap() + } + + /// what argument would this aggregate need in the plan? + fn column(&self) -> Arc { + match self { + Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), + Self::ColumnA(s) => expressions::col("a", s).unwrap(), + } + } + + /// What name would this aggregate produce in a plan? + pub fn column_name(&self) -> &'static str { + match self { + Self::CountStar => "COUNT(*)", + Self::ColumnA(_) => "COUNT(a)", + } + } + + /// What is the expected count? + pub fn expected_count(&self) -> i64 { + match self { + TestAggregate::CountStar => 3, + TestAggregate::ColumnA(_) => 2, + } + } +} diff --git a/datafusion/core/tests/physical_optimizer_integration.rs b/datafusion/core/tests/physical_optimizer_integration.rs new file mode 100644 index 000000000000..bbf4dcd2b799 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer_integration.rs @@ -0,0 +1,325 @@ +// 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. + +//! Tests for the physical optimizer + +use datafusion_common::config::ConfigOptions; +use datafusion_physical_optimizer::aggregate_statistics::AggregateStatistics; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::ExecutionPlan; +use std::sync::Arc; + +use datafusion::error::Result; +use datafusion::logical_expr::Operator; +use datafusion::prelude::SessionContext; +use datafusion::test_util::TestAggregate; +use datafusion_physical_plan::aggregates::PhysicalGroupBy; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::common; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::memory::MemoryExec; + +use arrow::array::Int32Array; +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::record_batch::RecordBatch; +use datafusion_common::cast::as_int64_array; +use datafusion_physical_expr::expressions::{self, cast}; +use datafusion_physical_plan::aggregates::AggregateMode; + +/// Mock data using a MemoryExec which has an exact count statistic +fn mock_data() -> Result> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![Some(1), Some(2), None])), + Arc::new(Int32Array::from(vec![Some(4), None, Some(6)])), + ], + )?; + + Ok(Arc::new(MemoryExec::try_new( + &[vec![batch]], + Arc::clone(&schema), + None, + )?)) +} + +/// Checks that the count optimization was applied and we still get the right result +async fn assert_count_optim_success( + plan: AggregateExec, + agg: TestAggregate, +) -> Result<()> { + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + let plan: Arc = Arc::new(plan); + + let optimized = + AggregateStatistics::new().optimize(Arc::clone(&plan), state.config_options())?; + + // A ProjectionExec is a sign that the count optimization was applied + assert!(optimized.as_any().is::()); + + // run both the optimized and nonoptimized plan + let optimized_result = + common::collect(optimized.execute(0, session_ctx.task_ctx())?).await?; + let nonoptimized_result = + common::collect(plan.execute(0, session_ctx.task_ctx())?).await?; + assert_eq!(optimized_result.len(), nonoptimized_result.len()); + + // and validate the results are the same and expected + assert_eq!(optimized_result.len(), 1); + check_batch(optimized_result.into_iter().next().unwrap(), &agg); + // check the non optimized one too to ensure types and names remain the same + assert_eq!(nonoptimized_result.len(), 1); + check_batch(nonoptimized_result.into_iter().next().unwrap(), &agg); + + Ok(()) +} + +fn check_batch(batch: RecordBatch, agg: &TestAggregate) { + let schema = batch.schema(); + let fields = schema.fields(); + assert_eq!(fields.len(), 1); + + let field = &fields[0]; + assert_eq!(field.name(), agg.column_name()); + assert_eq!(field.data_type(), &DataType::Int64); + // note that nullabiolity differs + + assert_eq!( + as_int64_array(batch.column(0)).unwrap().values(), + &[agg.expected_count()] + ); +} + +#[tokio::test] +async fn test_count_partial_direct_child() -> Result<()> { + // basic test case with the aggregation applied on a source with exact statistics + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_star(); + + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + source, + Arc::clone(&schema), + )?; + + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + Arc::new(partial_agg), + Arc::clone(&schema), + )?; + + assert_count_optim_success(final_agg, agg).await?; + + Ok(()) +} + +#[tokio::test] +async fn test_count_partial_with_nulls_direct_child() -> Result<()> { + // basic test case with the aggregation applied on a source with exact statistics + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_column(&schema); + + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + source, + Arc::clone(&schema), + )?; + + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + Arc::new(partial_agg), + Arc::clone(&schema), + )?; + + assert_count_optim_success(final_agg, agg).await?; + + Ok(()) +} + +#[tokio::test] +async fn test_count_partial_indirect_child() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_star(); + + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + source, + Arc::clone(&schema), + )?; + + // We introduce an intermediate optimization step between the partial and final aggregtator + let coalesce = CoalescePartitionsExec::new(Arc::new(partial_agg)); + + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + Arc::new(coalesce), + Arc::clone(&schema), + )?; + + assert_count_optim_success(final_agg, agg).await?; + + Ok(()) +} + +#[tokio::test] +async fn test_count_partial_with_nulls_indirect_child() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_column(&schema); + + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + source, + Arc::clone(&schema), + )?; + + // We introduce an intermediate optimization step between the partial and final aggregtator + let coalesce = CoalescePartitionsExec::new(Arc::new(partial_agg)); + + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + Arc::new(coalesce), + Arc::clone(&schema), + )?; + + assert_count_optim_success(final_agg, agg).await?; + + Ok(()) +} + +#[tokio::test] +async fn test_count_inexact_stat() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_star(); + + // adding a filter makes the statistics inexact + let filter = Arc::new(FilterExec::try_new( + expressions::binary( + expressions::col("a", &schema)?, + Operator::Gt, + cast(expressions::lit(1u32), &schema, DataType::Int32)?, + &schema, + )?, + source, + )?); + + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + filter, + Arc::clone(&schema), + )?; + + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + Arc::new(partial_agg), + Arc::clone(&schema), + )?; + + let conf = ConfigOptions::new(); + let optimized = AggregateStatistics::new().optimize(Arc::new(final_agg), &conf)?; + + // check that the original ExecutionPlan was not replaced + assert!(optimized.as_any().is::()); + + Ok(()) +} + +#[tokio::test] +async fn test_count_with_nulls_inexact_stat() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_column(&schema); + + // adding a filter makes the statistics inexact + let filter = Arc::new(FilterExec::try_new( + expressions::binary( + expressions::col("a", &schema)?, + Operator::Gt, + cast(expressions::lit(1u32), &schema, DataType::Int32)?, + &schema, + )?, + source, + )?); + + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + filter, + Arc::clone(&schema), + )?; + + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![agg.count_expr(&schema)], + vec![None], + Arc::new(partial_agg), + Arc::clone(&schema), + )?; + + let conf = ConfigOptions::new(); + let optimized = AggregateStatistics::new().optimize(Arc::new(final_agg), &conf)?; + + // check that the original ExecutionPlan was not replaced + assert!(optimized.as_any().is::()); + + Ok(()) +} diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 683a8e170ed4..65a70b673266 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -35,14 +35,14 @@ use datafusion_common::tree_node::{ use datafusion_common::utils::get_at_indices; use datafusion_common::{ internal_err, plan_datafusion_err, plan_err, Column, DFSchema, DFSchemaRef, Result, - ScalarValue, TableReference, + TableReference, }; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; /// The value to which `COUNT(*)` is expanded to in /// `COUNT()` expressions -pub const COUNT_STAR_EXPANSION: ScalarValue = ScalarValue::Int64(Some(1)); +pub use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; /// Recursively walk a list of expression trees, collecting the unique set of columns /// referenced in the expression diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs new file mode 100644 index 000000000000..0ce92df393aa --- /dev/null +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -0,0 +1,298 @@ +// 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. + +//! Utilizing exact statistics from sources to avoid scanning data +use std::sync::Arc; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::scalar::ScalarValue; +use datafusion_common::Result; +use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; + +use crate::PhysicalOptimizerRule; +use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; +use datafusion_physical_plan::udaf::AggregateFunctionExpr; + +/// Optimizer that uses available statistics for aggregate functions +#[derive(Default)] +pub struct AggregateStatistics {} + +impl AggregateStatistics { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for AggregateStatistics { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + if let Some(partial_agg_exec) = take_optimizable(&*plan) { + let partial_agg_exec = partial_agg_exec + .as_any() + .downcast_ref::() + .expect("take_optimizable() ensures that this is a AggregateExec"); + let stats = partial_agg_exec.input().statistics()?; + let mut projections = vec![]; + for expr in partial_agg_exec.aggr_expr() { + if let Some((non_null_rows, name)) = + take_optimizable_column_and_table_count(&**expr, &stats) + { + projections.push((expressions::lit(non_null_rows), name.to_owned())); + } else if let Some((min, name)) = take_optimizable_min(&**expr, &stats) { + projections.push((expressions::lit(min), name.to_owned())); + } else if let Some((max, name)) = take_optimizable_max(&**expr, &stats) { + projections.push((expressions::lit(max), name.to_owned())); + } else { + // TODO: we need all aggr_expr to be resolved (cf TODO fullres) + break; + } + } + + // TODO fullres: use statistics even if not all aggr_expr could be resolved + if projections.len() == partial_agg_exec.aggr_expr().len() { + // input can be entirely removed + Ok(Arc::new(ProjectionExec::try_new( + projections, + Arc::new(PlaceholderRowExec::new(plan.schema())), + )?)) + } else { + plan.map_children(|child| { + self.optimize(child, _config).map(Transformed::yes) + }) + .data() + } + } else { + plan.map_children(|child| self.optimize(child, _config).map(Transformed::yes)) + .data() + } + } + + fn name(&self) -> &str { + "aggregate_statistics" + } + + /// This rule will change the nullable properties of the schema, disable the schema check. + fn schema_check(&self) -> bool { + false + } +} + +/// assert if the node passed as argument is a final `AggregateExec` node that can be optimized: +/// - its child (with possible intermediate layers) is a partial `AggregateExec` node +/// - they both have no grouping expression +/// +/// If this is the case, return a ref to the partial `AggregateExec`, else `None`. +/// We would have preferred to return a casted ref to AggregateExec but the recursion requires +/// the `ExecutionPlan.children()` method that returns an owned reference. +fn take_optimizable(node: &dyn ExecutionPlan) -> Option> { + if let Some(final_agg_exec) = node.as_any().downcast_ref::() { + if !final_agg_exec.mode().is_first_stage() + && final_agg_exec.group_expr().is_empty() + { + let mut child = Arc::clone(final_agg_exec.input()); + loop { + if let Some(partial_agg_exec) = + child.as_any().downcast_ref::() + { + if partial_agg_exec.mode().is_first_stage() + && partial_agg_exec.group_expr().is_empty() + && partial_agg_exec.filter_expr().iter().all(|e| e.is_none()) + { + return Some(child); + } + } + if let [childrens_child] = child.children().as_slice() { + child = Arc::clone(childrens_child); + } else { + break; + } + } + } + } + None +} + +/// If this agg_expr is a count that can be exactly derived from the statistics, return it. +fn take_optimizable_column_and_table_count( + agg_expr: &dyn AggregateExpr, + stats: &Statistics, +) -> Option<(ScalarValue, String)> { + let col_stats = &stats.column_statistics; + if is_non_distinct_count(agg_expr) { + if let Precision::Exact(num_rows) = stats.num_rows { + let exprs = agg_expr.expressions(); + if exprs.len() == 1 { + // TODO optimize with exprs other than Column + if let Some(col_expr) = + exprs[0].as_any().downcast_ref::() + { + let current_val = &col_stats[col_expr.index()].null_count; + if let &Precision::Exact(val) = current_val { + return Some(( + ScalarValue::Int64(Some((num_rows - val) as i64)), + agg_expr.name().to_string(), + )); + } + } else if let Some(lit_expr) = + exprs[0].as_any().downcast_ref::() + { + if lit_expr.value() == &COUNT_STAR_EXPANSION { + return Some(( + ScalarValue::Int64(Some(num_rows as i64)), + agg_expr.name().to_string(), + )); + } + } + } + } + } + None +} + +/// If this agg_expr is a min that is exactly defined in the statistics, return it. +fn take_optimizable_min( + agg_expr: &dyn AggregateExpr, + stats: &Statistics, +) -> Option<(ScalarValue, String)> { + if let Precision::Exact(num_rows) = &stats.num_rows { + match *num_rows { + 0 => { + // MIN/MAX with 0 rows is always null + if is_min(agg_expr) { + if let Ok(min_data_type) = + ScalarValue::try_from(agg_expr.field().unwrap().data_type()) + { + return Some((min_data_type, agg_expr.name().to_string())); + } + } + } + value if value > 0 => { + let col_stats = &stats.column_statistics; + if is_min(agg_expr) { + let exprs = agg_expr.expressions(); + if exprs.len() == 1 { + // TODO optimize with exprs other than Column + if let Some(col_expr) = + exprs[0].as_any().downcast_ref::() + { + if let Precision::Exact(val) = + &col_stats[col_expr.index()].min_value + { + if !val.is_null() { + return Some(( + val.clone(), + agg_expr.name().to_string(), + )); + } + } + } + } + } + } + _ => {} + } + } + None +} + +/// If this agg_expr is a max that is exactly defined in the statistics, return it. +fn take_optimizable_max( + agg_expr: &dyn AggregateExpr, + stats: &Statistics, +) -> Option<(ScalarValue, String)> { + if let Precision::Exact(num_rows) = &stats.num_rows { + match *num_rows { + 0 => { + // MIN/MAX with 0 rows is always null + if is_max(agg_expr) { + if let Ok(max_data_type) = + ScalarValue::try_from(agg_expr.field().unwrap().data_type()) + { + return Some((max_data_type, agg_expr.name().to_string())); + } + } + } + value if value > 0 => { + let col_stats = &stats.column_statistics; + if is_max(agg_expr) { + let exprs = agg_expr.expressions(); + if exprs.len() == 1 { + // TODO optimize with exprs other than Column + if let Some(col_expr) = + exprs[0].as_any().downcast_ref::() + { + if let Precision::Exact(val) = + &col_stats[col_expr.index()].max_value + { + if !val.is_null() { + return Some(( + val.clone(), + agg_expr.name().to_string(), + )); + } + } + } + } + } + } + _ => {} + } + } + None +} + +// TODO: Move this check into AggregateUDFImpl +// https://github.com/apache/datafusion/issues/11153 +fn is_non_distinct_count(agg_expr: &dyn AggregateExpr) -> bool { + if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { + if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { + return true; + } + } + false +} + +// TODO: Move this check into AggregateUDFImpl +// https://github.com/apache/datafusion/issues/11153 +fn is_min(agg_expr: &dyn AggregateExpr) -> bool { + if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { + if agg_expr.fun().name().to_lowercase() == "min" { + return true; + } + } + false +} + +// TODO: Move this check into AggregateUDFImpl +// https://github.com/apache/datafusion/issues/11153 +fn is_max(agg_expr: &dyn AggregateExpr) -> bool { + if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { + if agg_expr.fun().name().to_lowercase() == "max" { + return true; + } + } + false +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 6b9df7cad5c8..8108493a0d3b 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -17,6 +17,7 @@ // Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 #![deny(clippy::clone_on_ref_ptr)] +pub mod aggregate_statistics; mod optimizer; pub mod output_requirements; From eb2b5fe74c7aef81cf55a7d6219cd15cc80e946b Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 6 Aug 2024 04:46:51 +0800 Subject: [PATCH 13/34] feat: Add support for cardinality function on maps (#11801) * feat: Add support for cardinality function on maps * chore: Fix prettier * feat: Add specialized signature for MapArray in ArrayFunctionSignature --- datafusion/expr/src/signature.rs | 6 +++ .../expr/src/type_coercion/functions.rs | 10 +++++ .../functions-nested/src/cardinality.rs | 39 +++++++++++++++---- datafusion/sqllogictest/test_files/map.slt | 9 +++++ docs/source/user-guide/expressions.md | 2 +- 5 files changed, 57 insertions(+), 9 deletions(-) diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr/src/signature.rs index b1cec3bad774..577c663142a1 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr/src/signature.rs @@ -145,6 +145,9 @@ pub enum ArrayFunctionSignature { /// The function takes a single argument that must be a List/LargeList/FixedSizeList /// or something that can be coerced to one of those types. Array, + /// Specialized Signature for MapArray + /// The function takes a single argument that must be a MapArray + MapArray, } impl std::fmt::Display for ArrayFunctionSignature { @@ -165,6 +168,9 @@ impl std::fmt::Display for ArrayFunctionSignature { ArrayFunctionSignature::Array => { write!(f, "array") } + ArrayFunctionSignature::MapArray => { + write!(f, "map_array") + } } } } diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index ef52a01e0598..66807c3f446c 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -378,6 +378,16 @@ fn get_valid_types( array(¤t_types[0]) .map_or_else(|| vec![vec![]], |array_type| vec![vec![array_type]]) } + ArrayFunctionSignature::MapArray => { + if current_types.len() != 1 { + return Ok(vec![vec![]]); + } + + match ¤t_types[0] { + DataType::Map(_, _) => vec![vec![current_types[0].clone()]], + _ => vec![vec![]], + } + } }, TypeSignature::Any(number) => { if current_types.len() != *number { diff --git a/datafusion/functions-nested/src/cardinality.rs b/datafusion/functions-nested/src/cardinality.rs index f6755c344768..ea07ac381aff 100644 --- a/datafusion/functions-nested/src/cardinality.rs +++ b/datafusion/functions-nested/src/cardinality.rs @@ -18,13 +18,18 @@ //! [`ScalarUDFImpl`] definitions for cardinality function. use crate::utils::make_scalar_function; -use arrow_array::{ArrayRef, GenericListArray, OffsetSizeTrait, UInt64Array}; +use arrow_array::{ + Array, ArrayRef, GenericListArray, MapArray, OffsetSizeTrait, UInt64Array, +}; use arrow_schema::DataType; -use arrow_schema::DataType::{FixedSizeList, LargeList, List, UInt64}; -use datafusion_common::cast::{as_large_list_array, as_list_array}; +use arrow_schema::DataType::{FixedSizeList, LargeList, List, Map, UInt64}; +use datafusion_common::cast::{as_large_list_array, as_list_array, as_map_array}; use datafusion_common::Result; use datafusion_common::{exec_err, plan_err}; -use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use datafusion_expr::{ + ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, + Volatility, +}; use std::any::Any; use std::sync::Arc; @@ -32,14 +37,20 @@ make_udf_expr_and_func!( Cardinality, cardinality, array, - "returns the total number of elements in the array.", + "returns the total number of elements in the array or map.", cardinality_udf ); impl Cardinality { pub fn new() -> Self { Self { - signature: Signature::array(Volatility::Immutable), + signature: Signature::one_of( + vec![ + TypeSignature::ArraySignature(ArrayFunctionSignature::Array), + TypeSignature::ArraySignature(ArrayFunctionSignature::MapArray), + ], + Volatility::Immutable, + ), aliases: vec![], } } @@ -64,9 +75,9 @@ impl ScalarUDFImpl for Cardinality { fn return_type(&self, arg_types: &[DataType]) -> Result { Ok(match arg_types[0] { - List(_) | LargeList(_) | FixedSizeList(_, _) => UInt64, + List(_) | LargeList(_) | FixedSizeList(_, _) | Map(_, _) => UInt64, _ => { - return plan_err!("The cardinality function can only accept List/LargeList/FixedSizeList."); + return plan_err!("The cardinality function can only accept List/LargeList/FixedSizeList/Map."); } }) } @@ -95,12 +106,24 @@ pub fn cardinality_inner(args: &[ArrayRef]) -> Result { let list_array = as_large_list_array(&args[0])?; generic_list_cardinality::(list_array) } + Map(_, _) => { + let map_array = as_map_array(&args[0])?; + generic_map_cardinality(map_array) + } other => { exec_err!("cardinality does not support type '{:?}'", other) } } } +fn generic_map_cardinality(array: &MapArray) -> Result { + let result: UInt64Array = array + .iter() + .map(|opt_arr| opt_arr.map(|arr| arr.len() as u64)) + .collect(); + Ok(Arc::new(result)) +} + fn generic_list_cardinality( array: &GenericListArray, ) -> Result { diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 11998eea9044..eb350c22bb5d 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -459,3 +459,12 @@ SELECT MAP { 'a': 1, 2: 3 }; # SELECT MAKE_MAP(1, null, 2, 33, 3, null)[2]; # ---- # 33 + +## cardinality + +# cardinality scalar function +query IIII +select cardinality(map([1, 2, 3], ['a', 'b', 'c'])), cardinality(MAP {'a': 1, 'b': null}), cardinality(MAP([],[])), + cardinality(MAP {'a': MAP {1:'a', 2:'b', 3:'c'}, 'b': MAP {2:'c', 4:'d'} }); +---- +3 2 0 2 diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 60036e440ffb..ad5a9cb75152 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -243,7 +243,7 @@ select log(-1), log(0), sqrt(-1); | array_except(array1, array2) | Returns an array of the elements that appear in the first array but not in the second. `array_except([1, 2, 3, 4], [5, 6, 3, 4]) -> [1, 2]` | | array_resize(array, size, value) | Resizes the list to contain size elements. Initializes new elements with value or empty if value is not set. `array_resize([1, 2, 3], 5, 0) -> [1, 2, 3, 0, 0]` | | array_sort(array, desc, null_first) | Returns sorted array. `array_sort([3, 1, 2, 5, 4]) -> [1, 2, 3, 4, 5]` | -| cardinality(array) | Returns the total number of elements in the array. `cardinality([[1, 2, 3], [4, 5, 6]]) -> 6` | +| cardinality(array/map) | Returns the total number of elements in the array or map. `cardinality([[1, 2, 3], [4, 5, 6]]) -> 6` | | make_array(value1, [value2 [, ...]]) | Returns an Arrow array using the specified input expressions. `make_array(1, 2, 3) -> [1, 2, 3]` | | range(start [, stop, step]) | Returns an Arrow array between start and stop with step. `SELECT range(2, 10, 3) -> [2, 5, 8]` | | string_to_array(array, delimiter, null_string) | Splits a `string` based on a `delimiter` and returns an array of parts. Any parts matching the optional `null_string` will be replaced with `NULL`. `string_to_array('abc#def#ghi', '#', ' ') -> ['abc', 'def', 'ghi']` | From a1645c4c1d6578d425e6a86648e32f13bca5cd43 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 5 Aug 2024 20:29:56 -0400 Subject: [PATCH 14/34] Minor: refactor probe check into function `should_skip_aggregation` (#11821) --- .../physical-plan/src/aggregates/row_hash.rs | 33 +++++++++++-------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 62ed79dad4aa..1b84befb0269 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -635,11 +635,7 @@ impl Stream for GroupedHashAggregateStream { ( if self.input_done { ExecutionState::Done - } else if self - .skip_aggregation_probe - .as_ref() - .is_some_and(|probe| probe.should_skip()) - { + } else if self.should_skip_aggregation() { ExecutionState::SkippingAggregation } else { ExecutionState::ReadingInput @@ -955,12 +951,13 @@ impl GroupedHashAggregateStream { Ok(()) } - // Updates skip aggregation probe state. - // In case stream has any spills, the probe is forcefully set to - // forbid aggregation skipping, and locked, since spilling resets - // total number of unique groups. - // - // Note: currently spilling is not supported for Partial aggregation + /// Updates skip aggregation probe state. + /// + /// In case stream has any spills, the probe is forcefully set to + /// forbid aggregation skipping, and locked, since spilling resets + /// total number of unique groups. + /// + /// Note: currently spilling is not supported for Partial aggregation fn update_skip_aggregation_probe(&mut self, input_rows: usize) { if let Some(probe) = self.skip_aggregation_probe.as_mut() { if !self.spill_state.spills.is_empty() { @@ -971,8 +968,8 @@ impl GroupedHashAggregateStream { }; } - // In case the probe indicates that aggregation may be - // skipped, forces stream to produce currently accumulated output. + /// In case the probe indicates that aggregation may be + /// skipped, forces stream to produce currently accumulated output. fn switch_to_skip_aggregation(&mut self) -> Result<()> { if let Some(probe) = self.skip_aggregation_probe.as_mut() { if probe.should_skip() { @@ -984,7 +981,15 @@ impl GroupedHashAggregateStream { Ok(()) } - // Transforms input batch to intermediate aggregate state, without grouping it + /// Returns true if the aggregation probe indicates that aggregation + /// should be skipped. + fn should_skip_aggregation(&self) -> bool { + self.skip_aggregation_probe + .as_ref() + .is_some_and(|probe| probe.should_skip()) + } + + /// Transforms input batch to intermediate aggregate state, without grouping it fn transform_to_states(&self, batch: RecordBatch) -> Result { let group_values = evaluate_group_by(&self.group_by, &batch)?; let input_values = evaluate_many(&self.aggregate_arguments, &batch)?; From 4a47dcbed03fdd4982f0e7a366a2db3af64c6c0c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 5 Aug 2024 23:39:10 -0400 Subject: [PATCH 15/34] Minor: move path_partition into `core_integration` (#11831) --- datafusion/core/tests/sql/mod.rs | 1 + datafusion/core/tests/{ => sql}/path_partition.rs | 0 2 files changed, 1 insertion(+) rename datafusion/core/tests/{ => sql}/path_partition.rs (100%) diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 995ce35c5bc2..dc9d04786021 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -60,6 +60,7 @@ pub mod aggregates; pub mod create_drop; pub mod explain_analyze; pub mod joins; +mod path_partition; pub mod select; mod sql_api; diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs similarity index 100% rename from datafusion/core/tests/path_partition.rs rename to datafusion/core/tests/sql/path_partition.rs From 537774ec1d96980f492ea46981dba5467c6d720a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 5 Aug 2024 23:53:19 -0400 Subject: [PATCH 16/34] Move optimizer integration tests to `core_integration` (#11830) --- datafusion/core/tests/core_integration.rs | 6 ++++++ .../mod.rs} | 0 .../aggregate_statistics.rs} | 0 .../core/tests/physical_optimizer/mod.rs | 18 ++++++++++++++++++ .../src/aggregate_statistics.rs | 2 ++ 5 files changed, 26 insertions(+) rename datafusion/core/tests/{optimizer_integration.rs => optimizer/mod.rs} (100%) rename datafusion/core/tests/{physical_optimizer_integration.rs => physical_optimizer/aggregate_statistics.rs} (100%) create mode 100644 datafusion/core/tests/physical_optimizer/mod.rs diff --git a/datafusion/core/tests/core_integration.rs b/datafusion/core/tests/core_integration.rs index deb5280388f1..79e5056e3cf5 100644 --- a/datafusion/core/tests/core_integration.rs +++ b/datafusion/core/tests/core_integration.rs @@ -36,6 +36,12 @@ mod memory_limit; /// Run all tests that are found in the `custom_sources_cases` directory mod custom_sources_cases; +/// Run all tests that are found in the `optimizer` directory +mod optimizer; + +/// Run all tests that are found in the `physical_optimizer` directory +mod physical_optimizer; + #[cfg(test)] #[ctor::ctor] fn init() { diff --git a/datafusion/core/tests/optimizer_integration.rs b/datafusion/core/tests/optimizer/mod.rs similarity index 100% rename from datafusion/core/tests/optimizer_integration.rs rename to datafusion/core/tests/optimizer/mod.rs diff --git a/datafusion/core/tests/physical_optimizer_integration.rs b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs similarity index 100% rename from datafusion/core/tests/physical_optimizer_integration.rs rename to datafusion/core/tests/physical_optimizer/aggregate_statistics.rs diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs new file mode 100644 index 000000000000..0ee89a3d213c --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -0,0 +1,18 @@ +// 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. + +mod aggregate_statistics; diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index 0ce92df393aa..66b250c5063b 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -296,3 +296,5 @@ fn is_max(agg_expr: &dyn AggregateExpr) -> bool { } false } + +// See tests in datafusion/core/tests/physical_optimizer From bf51babc9c7dc3e95476f84169646b4452fb9799 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 6 Aug 2024 13:05:55 +0800 Subject: [PATCH 17/34] Optionally create name of aggregate expression from expressions (#11776) * Optionally create name of aggregate expression from expressions * Fix test * name -> alias * keep trying to create aggr expr's display name first --- .../combine_partial_final_agg.rs | 4 +- datafusion/core/src/physical_planner.rs | 272 +----------------- datafusion/core/src/test_util/mod.rs | 2 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- datafusion/expr/src/expr.rs | 262 ++++++++++++++++- .../physical-expr-common/src/aggregate/mod.rs | 40 ++- .../physical-plan/src/aggregates/mod.rs | 52 ++-- datafusion/physical-plan/src/windows/mod.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 20 +- 10 files changed, 337 insertions(+), 321 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 6f3274820c8c..843efcc7b0d2 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -280,7 +280,7 @@ mod tests { ) -> Arc { AggregateExprBuilder::new(count_udaf(), vec![expr]) .schema(Arc::new(schema.clone())) - .name(name) + .alias(name) .build() .unwrap() } @@ -364,7 +364,7 @@ mod tests { vec![ AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) - .name("Sum(b)") + .alias("Sum(b)") .build() .unwrap(), ]; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 378a892111c5..65cdbf9fe62c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -19,7 +19,6 @@ use std::borrow::Cow; use std::collections::HashMap; -use std::fmt::Write; use std::sync::Arc; use crate::datasource::file_format::file_type_to_format; @@ -74,11 +73,9 @@ use datafusion_common::{ }; use datafusion_expr::dml::CopyTo; use datafusion_expr::expr::{ - self, AggregateFunction, Alias, Between, BinaryExpr, Cast, GroupingSet, InList, Like, - TryCast, WindowFunction, + self, physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, }; use datafusion_expr::expr_rewriter::unnormalize_cols; -use datafusion_expr::expr_vec_fmt; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, StringifiedPlan, @@ -97,265 +94,6 @@ use log::{debug, trace}; use sqlparser::ast::NullTreatment; use tokio::sync::Mutex; -fn create_function_physical_name( - fun: &str, - distinct: bool, - args: &[Expr], - order_by: Option<&Vec>, -) -> Result { - let names: Vec = args - .iter() - .map(|e| create_physical_name(e, false)) - .collect::>()?; - - let distinct_str = match distinct { - true => "DISTINCT ", - false => "", - }; - - let phys_name = format!("{}({}{})", fun, distinct_str, names.join(",")); - - Ok(order_by - .map(|order_by| format!("{} ORDER BY [{}]", phys_name, expr_vec_fmt!(order_by))) - .unwrap_or(phys_name)) -} - -fn physical_name(e: &Expr) -> Result { - create_physical_name(e, true) -} - -fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { - match e { - Expr::Unnest(_) => { - internal_err!( - "Expr::Unnest should have been converted to LogicalPlan::Unnest" - ) - } - Expr::Column(c) => { - if is_first_expr { - Ok(c.name.clone()) - } else { - Ok(c.flat_name()) - } - } - Expr::Alias(Alias { name, .. }) => Ok(name.clone()), - Expr::ScalarVariable(_, variable_names) => Ok(variable_names.join(".")), - Expr::Literal(value) => Ok(format!("{value:?}")), - Expr::BinaryExpr(BinaryExpr { left, op, right }) => { - let left = create_physical_name(left, false)?; - let right = create_physical_name(right, false)?; - Ok(format!("{left} {op} {right}")) - } - Expr::Case(case) => { - let mut name = "CASE ".to_string(); - if let Some(e) = &case.expr { - let _ = write!(name, "{} ", create_physical_name(e, false)?); - } - for (w, t) in &case.when_then_expr { - let _ = write!( - name, - "WHEN {} THEN {} ", - create_physical_name(w, false)?, - create_physical_name(t, false)? - ); - } - if let Some(e) = &case.else_expr { - let _ = write!(name, "ELSE {} ", create_physical_name(e, false)?); - } - name += "END"; - Ok(name) - } - Expr::Cast(Cast { expr, .. }) => { - // CAST does not change the expression name - create_physical_name(expr, false) - } - Expr::TryCast(TryCast { expr, .. }) => { - // CAST does not change the expression name - create_physical_name(expr, false) - } - Expr::Not(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("NOT {expr}")) - } - Expr::Negative(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("(- {expr})")) - } - Expr::IsNull(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS NULL")) - } - Expr::IsNotNull(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS NOT NULL")) - } - Expr::IsTrue(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS TRUE")) - } - Expr::IsFalse(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS FALSE")) - } - Expr::IsUnknown(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS UNKNOWN")) - } - Expr::IsNotTrue(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS NOT TRUE")) - } - Expr::IsNotFalse(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS NOT FALSE")) - } - Expr::IsNotUnknown(expr) => { - let expr = create_physical_name(expr, false)?; - Ok(format!("{expr} IS NOT UNKNOWN")) - } - Expr::ScalarFunction(fun) => fun.func.display_name(&fun.args), - Expr::WindowFunction(WindowFunction { - fun, - args, - order_by, - .. - }) => { - create_function_physical_name(&fun.to_string(), false, args, Some(order_by)) - } - Expr::AggregateFunction(AggregateFunction { - func, - distinct, - args, - filter: _, - order_by, - null_treatment: _, - }) => { - create_function_physical_name(func.name(), *distinct, args, order_by.as_ref()) - } - Expr::GroupingSet(grouping_set) => match grouping_set { - GroupingSet::Rollup(exprs) => Ok(format!( - "ROLLUP ({})", - exprs - .iter() - .map(|e| create_physical_name(e, false)) - .collect::>>()? - .join(", ") - )), - GroupingSet::Cube(exprs) => Ok(format!( - "CUBE ({})", - exprs - .iter() - .map(|e| create_physical_name(e, false)) - .collect::>>()? - .join(", ") - )), - GroupingSet::GroupingSets(lists_of_exprs) => { - let mut strings = vec![]; - for exprs in lists_of_exprs { - let exprs_str = exprs - .iter() - .map(|e| create_physical_name(e, false)) - .collect::>>()? - .join(", "); - strings.push(format!("({exprs_str})")); - } - Ok(format!("GROUPING SETS ({})", strings.join(", "))) - } - }, - - Expr::InList(InList { - expr, - list, - negated, - }) => { - let expr = create_physical_name(expr, false)?; - let list = list.iter().map(|expr| create_physical_name(expr, false)); - if *negated { - Ok(format!("{expr} NOT IN ({list:?})")) - } else { - Ok(format!("{expr} IN ({list:?})")) - } - } - Expr::Exists { .. } => { - not_impl_err!("EXISTS is not yet supported in the physical plan") - } - Expr::InSubquery(_) => { - not_impl_err!("IN subquery is not yet supported in the physical plan") - } - Expr::ScalarSubquery(_) => { - not_impl_err!("Scalar subqueries are not yet supported in the physical plan") - } - Expr::Between(Between { - expr, - negated, - low, - high, - }) => { - let expr = create_physical_name(expr, false)?; - let low = create_physical_name(low, false)?; - let high = create_physical_name(high, false)?; - if *negated { - Ok(format!("{expr} NOT BETWEEN {low} AND {high}")) - } else { - Ok(format!("{expr} BETWEEN {low} AND {high}")) - } - } - Expr::Like(Like { - negated, - expr, - pattern, - escape_char, - case_insensitive, - }) => { - let expr = create_physical_name(expr, false)?; - let pattern = create_physical_name(pattern, false)?; - let op_name = if *case_insensitive { "ILIKE" } else { "LIKE" }; - let escape = if let Some(char) = escape_char { - format!("CHAR '{char}'") - } else { - "".to_string() - }; - if *negated { - Ok(format!("{expr} NOT {op_name} {pattern}{escape}")) - } else { - Ok(format!("{expr} {op_name} {pattern}{escape}")) - } - } - Expr::SimilarTo(Like { - negated, - expr, - pattern, - escape_char, - case_insensitive: _, - }) => { - let expr = create_physical_name(expr, false)?; - let pattern = create_physical_name(pattern, false)?; - let escape = if let Some(char) = escape_char { - format!("CHAR '{char}'") - } else { - "".to_string() - }; - if *negated { - Ok(format!("{expr} NOT SIMILAR TO {pattern}{escape}")) - } else { - Ok(format!("{expr} SIMILAR TO {pattern}{escape}")) - } - } - Expr::Sort { .. } => { - internal_err!("Create physical name does not support sort expression") - } - Expr::Wildcard { .. } => { - internal_err!("Create physical name does not support wildcard") - } - Expr::Placeholder(_) => { - internal_err!("Create physical name does not support placeholder") - } - Expr::OuterReferenceColumn(_, _) => { - internal_err!("Create physical name does not support OuterReferenceColumn") - } - } -} - /// Physical query planner that converts a `LogicalPlan` to an /// `ExecutionPlan` suitable for execution. #[async_trait] @@ -1807,7 +1545,7 @@ type AggregateExprWithOptionalArgs = ( /// Create an aggregate expression with a name from a logical expression pub fn create_aggregate_expr_with_name_and_maybe_filter( e: &Expr, - name: impl Into, + name: Option, logical_input_schema: &DFSchema, _physical_input_schema: &Schema, execution_props: &ExecutionProps, @@ -1881,9 +1619,9 @@ pub fn create_aggregate_expr_and_maybe_filter( ) -> Result { // unpack (nested) aliased logical expressions, e.g. "sum(col) as total" let (name, e) = match e { - Expr::Alias(Alias { expr, name, .. }) => (name.clone(), expr.as_ref()), - Expr::AggregateFunction(_) => (e.display_name().unwrap_or(physical_name(e)?), e), - _ => (physical_name(e)?, e), + Expr::Alias(Alias { expr, name, .. }) => (Some(name.clone()), expr.as_ref()), + Expr::AggregateFunction(_) => (e.display_name().ok(), e), + _ => (None, e), }; create_aggregate_expr_with_name_and_maybe_filter( diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 6eb82dece31c..9610a7f20364 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -432,7 +432,7 @@ impl TestAggregate { pub fn count_expr(&self, schema: &Schema) -> Arc { AggregateExprBuilder::new(count_udaf(), vec![self.column()]) .schema(Arc::new(schema.clone())) - .name(self.column_name()) + .alias(self.column_name()) .build() .unwrap() } diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 6f286c9aeba1..4cecb0b69335 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -107,7 +107,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str vec![ AggregateExprBuilder::new(sum_udaf(), vec![col("d", &schema).unwrap()]) .schema(Arc::clone(&schema)) - .name("sum1") + .alias("sum1") .build() .unwrap(), ]; diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 1a51c181f49f..edf45a244e1f 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -38,7 +38,8 @@ use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; use datafusion_common::{ - internal_err, plan_err, Column, DFSchema, Result, ScalarValue, TableReference, + internal_err, not_impl_err, plan_err, Column, DFSchema, Result, ScalarValue, + TableReference, }; use sqlparser::ast::NullTreatment; @@ -2277,6 +2278,265 @@ fn write_names_join(w: &mut W, exprs: &[Expr], sep: &str) -> Result<() Ok(()) } +pub fn create_function_physical_name( + fun: &str, + distinct: bool, + args: &[Expr], + order_by: Option<&Vec>, +) -> Result { + let names: Vec = args + .iter() + .map(|e| create_physical_name(e, false)) + .collect::>()?; + + let distinct_str = match distinct { + true => "DISTINCT ", + false => "", + }; + + let phys_name = format!("{}({}{})", fun, distinct_str, names.join(",")); + + Ok(order_by + .map(|order_by| format!("{} ORDER BY [{}]", phys_name, expr_vec_fmt!(order_by))) + .unwrap_or(phys_name)) +} + +pub fn physical_name(e: &Expr) -> Result { + create_physical_name(e, true) +} + +fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { + match e { + Expr::Unnest(_) => { + internal_err!( + "Expr::Unnest should have been converted to LogicalPlan::Unnest" + ) + } + Expr::Column(c) => { + if is_first_expr { + Ok(c.name.clone()) + } else { + Ok(c.flat_name()) + } + } + Expr::Alias(Alias { name, .. }) => Ok(name.clone()), + Expr::ScalarVariable(_, variable_names) => Ok(variable_names.join(".")), + Expr::Literal(value) => Ok(format!("{value:?}")), + Expr::BinaryExpr(BinaryExpr { left, op, right }) => { + let left = create_physical_name(left, false)?; + let right = create_physical_name(right, false)?; + Ok(format!("{left} {op} {right}")) + } + Expr::Case(case) => { + let mut name = "CASE ".to_string(); + if let Some(e) = &case.expr { + let _ = write!(name, "{} ", create_physical_name(e, false)?); + } + for (w, t) in &case.when_then_expr { + let _ = write!( + name, + "WHEN {} THEN {} ", + create_physical_name(w, false)?, + create_physical_name(t, false)? + ); + } + if let Some(e) = &case.else_expr { + let _ = write!(name, "ELSE {} ", create_physical_name(e, false)?); + } + name += "END"; + Ok(name) + } + Expr::Cast(Cast { expr, .. }) => { + // CAST does not change the expression name + create_physical_name(expr, false) + } + Expr::TryCast(TryCast { expr, .. }) => { + // CAST does not change the expression name + create_physical_name(expr, false) + } + Expr::Not(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("NOT {expr}")) + } + Expr::Negative(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("(- {expr})")) + } + Expr::IsNull(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS NULL")) + } + Expr::IsNotNull(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS NOT NULL")) + } + Expr::IsTrue(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS TRUE")) + } + Expr::IsFalse(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS FALSE")) + } + Expr::IsUnknown(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS UNKNOWN")) + } + Expr::IsNotTrue(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS NOT TRUE")) + } + Expr::IsNotFalse(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS NOT FALSE")) + } + Expr::IsNotUnknown(expr) => { + let expr = create_physical_name(expr, false)?; + Ok(format!("{expr} IS NOT UNKNOWN")) + } + Expr::ScalarFunction(fun) => fun.func.display_name(&fun.args), + Expr::WindowFunction(WindowFunction { + fun, + args, + order_by, + .. + }) => { + create_function_physical_name(&fun.to_string(), false, args, Some(order_by)) + } + Expr::AggregateFunction(AggregateFunction { + func, + distinct, + args, + filter: _, + order_by, + null_treatment: _, + }) => { + create_function_physical_name(func.name(), *distinct, args, order_by.as_ref()) + } + Expr::GroupingSet(grouping_set) => match grouping_set { + GroupingSet::Rollup(exprs) => Ok(format!( + "ROLLUP ({})", + exprs + .iter() + .map(|e| create_physical_name(e, false)) + .collect::>>()? + .join(", ") + )), + GroupingSet::Cube(exprs) => Ok(format!( + "CUBE ({})", + exprs + .iter() + .map(|e| create_physical_name(e, false)) + .collect::>>()? + .join(", ") + )), + GroupingSet::GroupingSets(lists_of_exprs) => { + let mut strings = vec![]; + for exprs in lists_of_exprs { + let exprs_str = exprs + .iter() + .map(|e| create_physical_name(e, false)) + .collect::>>()? + .join(", "); + strings.push(format!("({exprs_str})")); + } + Ok(format!("GROUPING SETS ({})", strings.join(", "))) + } + }, + + Expr::InList(InList { + expr, + list, + negated, + }) => { + let expr = create_physical_name(expr, false)?; + let list = list.iter().map(|expr| create_physical_name(expr, false)); + if *negated { + Ok(format!("{expr} NOT IN ({list:?})")) + } else { + Ok(format!("{expr} IN ({list:?})")) + } + } + Expr::Exists { .. } => { + not_impl_err!("EXISTS is not yet supported in the physical plan") + } + Expr::InSubquery(_) => { + not_impl_err!("IN subquery is not yet supported in the physical plan") + } + Expr::ScalarSubquery(_) => { + not_impl_err!("Scalar subqueries are not yet supported in the physical plan") + } + Expr::Between(Between { + expr, + negated, + low, + high, + }) => { + let expr = create_physical_name(expr, false)?; + let low = create_physical_name(low, false)?; + let high = create_physical_name(high, false)?; + if *negated { + Ok(format!("{expr} NOT BETWEEN {low} AND {high}")) + } else { + Ok(format!("{expr} BETWEEN {low} AND {high}")) + } + } + Expr::Like(Like { + negated, + expr, + pattern, + escape_char, + case_insensitive, + }) => { + let expr = create_physical_name(expr, false)?; + let pattern = create_physical_name(pattern, false)?; + let op_name = if *case_insensitive { "ILIKE" } else { "LIKE" }; + let escape = if let Some(char) = escape_char { + format!("CHAR '{char}'") + } else { + "".to_string() + }; + if *negated { + Ok(format!("{expr} NOT {op_name} {pattern}{escape}")) + } else { + Ok(format!("{expr} {op_name} {pattern}{escape}")) + } + } + Expr::SimilarTo(Like { + negated, + expr, + pattern, + escape_char, + case_insensitive: _, + }) => { + let expr = create_physical_name(expr, false)?; + let pattern = create_physical_name(pattern, false)?; + let escape = if let Some(char) = escape_char { + format!("CHAR '{char}'") + } else { + "".to_string() + }; + if *negated { + Ok(format!("{expr} NOT SIMILAR TO {pattern}{escape}")) + } else { + Ok(format!("{expr} SIMILAR TO {pattern}{escape}")) + } + } + Expr::Sort { .. } => { + internal_err!("Create physical name does not support sort expression") + } + Expr::Wildcard { .. } => { + internal_err!("Create physical name does not support wildcard") + } + Expr::Placeholder(_) => { + internal_err!("Create physical name does not support placeholder") + } + Expr::OuterReferenceColumn(_, _) => { + internal_err!("Create physical name does not support OuterReferenceColumn") + } + } +} + #[cfg(test)] mod test { use crate::expr_fn::col; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 665cdd708329..350023352b12 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -22,6 +22,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::exec_err; use datafusion_common::{internal_err, not_impl_err, DFSchema, Result}; +use datafusion_expr::expr::create_function_physical_name; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::utils::AggregateOrderSensitivity; @@ -67,7 +68,7 @@ pub fn create_aggregate_expr( sort_exprs: &[Expr], ordering_req: &[PhysicalSortExpr], schema: &Schema, - name: impl Into, + name: Option, ignore_nulls: bool, is_distinct: bool, ) -> Result> { @@ -77,7 +78,9 @@ pub fn create_aggregate_expr( builder = builder.order_by(ordering_req.to_vec()); builder = builder.logical_exprs(input_exprs.to_vec()); builder = builder.schema(Arc::new(schema.clone())); - builder = builder.name(name); + if let Some(name) = name { + builder = builder.alias(name); + } if ignore_nulls { builder = builder.ignore_nulls(); @@ -98,7 +101,7 @@ pub fn create_aggregate_expr_with_dfschema( sort_exprs: &[Expr], ordering_req: &[PhysicalSortExpr], dfschema: &DFSchema, - name: impl Into, + alias: Option, ignore_nulls: bool, is_distinct: bool, is_reversed: bool, @@ -111,7 +114,9 @@ pub fn create_aggregate_expr_with_dfschema( builder = builder.dfschema(dfschema.clone()); let schema: Schema = dfschema.into(); builder = builder.schema(Arc::new(schema)); - builder = builder.name(name); + if let Some(alias) = alias { + builder = builder.alias(alias); + } if ignore_nulls { builder = builder.ignore_nulls(); @@ -137,7 +142,7 @@ pub struct AggregateExprBuilder { args: Vec>, /// Logical expressions of the aggregate function, it will be deprecated in logical_args: Vec, - name: String, + alias: Option, /// Arrow Schema for the aggregate function schema: SchemaRef, /// Datafusion Schema for the aggregate function @@ -160,7 +165,7 @@ impl AggregateExprBuilder { fun, args, logical_args: vec![], - name: String::new(), + alias: None, schema: Arc::new(Schema::empty()), dfschema: DFSchema::empty(), sort_exprs: vec![], @@ -176,7 +181,7 @@ impl AggregateExprBuilder { fun, args, logical_args, - name, + alias, schema, dfschema, sort_exprs, @@ -213,6 +218,19 @@ impl AggregateExprBuilder { )?; let data_type = fun.return_type(&input_exprs_types)?; + let name = match alias { + None => create_function_physical_name( + fun.name(), + is_distinct, + &logical_args, + if sort_exprs.is_empty() { + None + } else { + Some(&sort_exprs) + }, + )?, + Some(alias) => alias, + }; Ok(Arc::new(AggregateFunctionExpr { fun: Arc::unwrap_or_clone(fun), @@ -232,8 +250,8 @@ impl AggregateExprBuilder { })) } - pub fn name(mut self, name: impl Into) -> Self { - self.name = name.into(); + pub fn alias(mut self, alias: impl Into) -> Self { + self.alias = Some(alias.into()); self } @@ -680,7 +698,7 @@ impl AggregateExpr for AggregateFunctionExpr { &self.sort_exprs, &self.ordering_req, &self.dfschema, - self.name(), + Some(self.name().to_string()), self.ignore_nulls, self.is_distinct, self.is_reversed, @@ -721,7 +739,7 @@ impl AggregateExpr for AggregateFunctionExpr { &reverse_sort_exprs, &reverse_ordering_req, &self.dfschema, - name, + Some(name), self.ignore_nulls, self.is_distinct, !self.is_reversed, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 8941418c12e1..d72da9b30049 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1362,7 +1362,7 @@ mod tests { let aggregates = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1i8)]) .schema(Arc::clone(&input_schema)) - .name("COUNT(1)") + .alias("COUNT(1)") .logical_exprs(vec![datafusion_expr::lit(1i8)]) .build()?]; @@ -1507,7 +1507,7 @@ mod tests { vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) .schema(Arc::clone(&input_schema)) - .name("AVG(b)") + .alias("AVG(b)") .build()?, ]; @@ -1803,7 +1803,7 @@ mod tests { fn test_median_agg_expr(schema: SchemaRef) -> Result> { AggregateExprBuilder::new(median_udaf(), vec![col("a", &schema)?]) .schema(schema) - .name("MEDIAN(a)") + .alias("MEDIAN(a)") .build() } @@ -1834,7 +1834,7 @@ mod tests { vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) .schema(Arc::clone(&input_schema)) - .name("AVG(b)") + .alias("AVG(b)") .build()?, ]; @@ -1894,7 +1894,7 @@ mod tests { vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) .schema(Arc::clone(&schema)) - .name("AVG(a)") + .alias("AVG(a)") .build()?, ]; @@ -1934,7 +1934,7 @@ mod tests { vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) - .name("AVG(b)") + .alias("AVG(b)") .build()?, ]; @@ -2002,7 +2002,7 @@ mod tests { &sort_exprs, &ordering_req, dfschema, - "FIRST_VALUE(b)", + None, false, false, false, @@ -2034,7 +2034,7 @@ mod tests { &sort_exprs, &ordering_req, dfschema, - "LAST_VALUE(b)", + None, false, false, false, @@ -2130,24 +2130,24 @@ mod tests { let result = crate::collect(aggregate_final, task_ctx).await?; if is_first_acc { let expected = [ - "+---+----------------+", - "| a | FIRST_VALUE(b) |", - "+---+----------------+", - "| 2 | 0.0 |", - "| 3 | 1.0 |", - "| 4 | 3.0 |", - "+---+----------------+", + "+---+--------------------------------------------+", + "| a | first_value(b) ORDER BY [b ASC NULLS LAST] |", + "+---+--------------------------------------------+", + "| 2 | 0.0 |", + "| 3 | 1.0 |", + "| 4 | 3.0 |", + "+---+--------------------------------------------+", ]; assert_batches_eq!(expected, &result); } else { let expected = [ - "+---+---------------+", - "| a | LAST_VALUE(b) |", - "+---+---------------+", - "| 2 | 3.0 |", - "| 3 | 5.0 |", - "| 4 | 6.0 |", - "+---+---------------+", + "+---+-------------------------------------------+", + "| a | last_value(b) ORDER BY [b ASC NULLS LAST] |", + "+---+-------------------------------------------+", + "| 2 | 3.0 |", + "| 3 | 5.0 |", + "| 4 | 6.0 |", + "+---+-------------------------------------------+", ]; assert_batches_eq!(expected, &result); }; @@ -2267,7 +2267,7 @@ mod tests { &sort_exprs, &ordering_req, &test_df_schema, - "array_agg", + None, false, false, false, @@ -2363,7 +2363,7 @@ mod tests { let aggregates: Vec> = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1)]) .schema(Arc::clone(&schema)) - .name("1") + .alias("1") .build()?]; let input_batches = (0..4) @@ -2427,7 +2427,7 @@ mod tests { &[], &[], &df_schema, - "COUNT(val)", + Some("COUNT(val)".to_string()), false, false, false, @@ -2515,7 +2515,7 @@ mod tests { &[], &[], &df_schema, - "COUNT(val)", + Some("COUNT(val)".to_string()), false, false, false, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 65cef28efc45..b41f3ad71bb8 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -128,7 +128,7 @@ pub fn create_window_expr( let aggregate = AggregateExprBuilder::new(Arc::clone(fun), args.to_vec()) .schema(Arc::new(input_schema.clone())) - .name(name) + .alias(name) .order_by(order_by.to_vec()) .sort_exprs(sort_exprs) .with_ignore_nulls(ignore_nulls) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index aefa1d87a278..59db791c7595 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -492,7 +492,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { // https://github.com/apache/datafusion/issues/11804 AggregateExprBuilder::new(agg_udf, input_phy_expr) .schema(Arc::clone(&physical_schema)) - .name(name) + .alias(name) .with_ignore_nulls(agg_node.ignore_nulls) .with_distinct(agg_node.distinct) .build() diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 0e2bc9cbb3e2..712182791b0b 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -296,7 +296,7 @@ fn roundtrip_window() -> Result<()> { vec![cast(col("b", &schema)?, &schema, DataType::Float64)?], ) .schema(Arc::clone(&schema)) - .name("avg(b)") + .alias("avg(b)") .build()?, &[], &[], @@ -312,7 +312,7 @@ fn roundtrip_window() -> Result<()> { let args = vec![cast(col("a", &schema)?, &schema, DataType::Float64)?]; let sum_expr = AggregateExprBuilder::new(sum_udaf(), args) .schema(Arc::clone(&schema)) - .name("SUM(a) RANGE BETWEEN CURRENT ROW AND UNBOUNDED PRECEEDING") + .alias("SUM(a) RANGE BETWEEN CURRENT ROW AND UNBOUNDED PRECEEDING") .build()?; let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( @@ -346,17 +346,17 @@ fn rountrip_aggregate() -> Result<()> { let avg_expr = AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) - .name("AVG(b)") + .alias("AVG(b)") .build()?; let nth_expr = AggregateExprBuilder::new(nth_value_udaf(), vec![col("b", &schema)?, lit(1u64)]) .schema(Arc::clone(&schema)) - .name("NTH_VALUE(b, 1)") + .alias("NTH_VALUE(b, 1)") .build()?; let str_agg_expr = AggregateExprBuilder::new(string_agg_udaf(), vec![col("b", &schema)?, lit(1u64)]) .schema(Arc::clone(&schema)) - .name("NTH_VALUE(b, 1)") + .alias("NTH_VALUE(b, 1)") .build()?; let test_cases: Vec>> = vec![ @@ -396,7 +396,7 @@ fn rountrip_aggregate_with_limit() -> Result<()> { vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) - .name("AVG(b)") + .alias("AVG(b)") .build()?, ]; @@ -463,7 +463,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { vec![ AggregateExprBuilder::new(Arc::new(udaf), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) - .name("example_agg") + .alias("example_agg") .build()?, ]; @@ -914,7 +914,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { vec![udf_expr.clone() as Arc], ) .schema(schema.clone()) - .name("max") + .alias("max") .build()?; let window = Arc::new(WindowAggExec::try_new( @@ -965,7 +965,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { let aggr_expr = AggregateExprBuilder::new(Arc::clone(&udaf), aggr_args.clone()) .schema(Arc::clone(&schema)) - .name("aggregate_udf") + .alias("aggregate_udf") .build()?; let filter = Arc::new(FilterExec::try_new( @@ -990,7 +990,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { let aggr_expr = AggregateExprBuilder::new(udaf, aggr_args.clone()) .schema(Arc::clone(&schema)) - .name("aggregate_udf") + .alias("aggregate_udf") .distinct() .ignore_nulls() .build()?; From d8bc7e2e6c50b26509b7fe7ef8915aa3010afc92 Mon Sep 17 00:00:00 2001 From: Kezhu Wang Date: Tue, 6 Aug 2024 13:38:31 +0800 Subject: [PATCH 18/34] Bump deprecated version of SessionState::new_with_config_rt to 41.0.0 (#11839) SessionState::new_with_config_rt was deprecated in favor of SessionStateBuilder in #11403 which is not shipped in 40.x. --- datafusion/core/src/execution/session_state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index ccad0240fddb..0a057d6f1417 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -249,7 +249,7 @@ impl Session for SessionState { impl SessionState { /// Returns new [`SessionState`] using the provided /// [`SessionConfig`] and [`RuntimeEnv`]. - #[deprecated(since = "40.0.0", note = "Use SessionStateBuilder")] + #[deprecated(since = "41.0.0", note = "Use SessionStateBuilder")] pub fn new_with_config_rt(config: SessionConfig, runtime: Arc) -> Self { SessionStateBuilder::new() .with_config(config) From 1c98e6e640090600d36ce51f87919fea7b6eeed2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 6 Aug 2024 03:22:52 -0400 Subject: [PATCH 19/34] Fix partial aggregation skipping with Decimal aggregators (#11833) * Fix partial aggregation skipping for Decimal * fix bug --- .../aggregate/groups_accumulator/prim_op.rs | 3 ++- .../test_files/aggregate_skip_partial.slt | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs index 8d69646bd422..acf1ae525c79 100644 --- a/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/physical-expr-common/src/aggregate/groups_accumulator/prim_op.rs @@ -184,7 +184,8 @@ where "initial_values underlying buffer must not be shared" ) })? - .map_err(DataFusionError::from)?; + .map_err(DataFusionError::from)? + .with_data_type(self.data_type.clone()); Ok(vec![Arc::new(state_values)]) } diff --git a/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt b/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt index 65efc24ec037..6c0cf5f800d8 100644 --- a/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt +++ b/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt @@ -322,3 +322,29 @@ FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; 3 109 211 2.80575042963 2.80632930994 4 -171 56 2.10740506649 1.939846396446 5 -86 -76 1.8741710186 1.600569307804 + + +statement ok +DROP TABLE aggregate_test_100_null; + +# Test for aggregate functions with different intermediate types +# Need more than 10 values to trigger skipping +statement ok +CREATE TABLE decimal_table(i int, d decimal(10,3)) as +VALUES (1, 1.1), (2, 2.2), (3, 3.3), (2, 4.4), (1, 5.5); + +statement ok +CREATE TABLE t(id int) as values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10); + +query IR +SELECT i, sum(d) +FROM decimal_table CROSS JOIN t +GROUP BY i +ORDER BY i; +---- +1 66 +2 66 +3 33 + +statement ok +DROP TABLE decimal_table; From 3d76aa25e4830ef8da42fae17453d8d1b8e66d4e Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Tue, 6 Aug 2024 03:23:59 -0700 Subject: [PATCH 20/34] feat: support `Utf8View` type in `starts_with` function (#11787) * feat: support `Utf8View` for `starts_with` * style: clippy * simplify string view handling * fix: allow utf8 and largeutf8 to be cast into utf8view * fix: fix test * Apply suggestions from code review Co-authored-by: Yongting You <2010youy01@gmail.com> Co-authored-by: Andrew Lamb * style: fix format * feat: add addiontal tests * tests: improve tests * fix: fix null case * tests: one more null test * Test comments and execution tests --------- Co-authored-by: Yongting You <2010youy01@gmail.com> Co-authored-by: Andrew Lamb --- datafusion/expr/src/expr_schema.rs | 1 + .../expr/src/type_coercion/functions.rs | 16 ++++ .../functions/src/string/starts_with.rs | 92 +++++++++++++++---- .../sqllogictest/test_files/string_view.slt | 70 +++++++++++++- 4 files changed, 158 insertions(+), 21 deletions(-) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 676903d59a07..9faeb8aed506 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -148,6 +148,7 @@ impl ExprSchemable for Expr { .iter() .map(|e| e.get_type(schema)) .collect::>>()?; + // verify that function is invoked with correct number and type of arguments as defined in `TypeSignature` data_types_with_scalar_udf(&arg_data_types, func).map_err(|err| { plan_datafusion_err!( diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 66807c3f446c..4f2776516d3e 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -583,6 +583,10 @@ fn coerced_from<'a>( (Interval(_), _) if matches!(type_from, Utf8 | LargeUtf8) => { Some(type_into.clone()) } + // We can go into a Utf8View from a Utf8 or LargeUtf8 + (Utf8View, _) if matches!(type_from, Utf8 | LargeUtf8 | Null) => { + Some(type_into.clone()) + } // Any type can be coerced into strings (Utf8 | LargeUtf8, _) => Some(type_into.clone()), (Null, _) if can_cast_types(type_from, type_into) => Some(type_into.clone()), @@ -646,6 +650,18 @@ mod tests { use super::*; use arrow::datatypes::Field; + #[test] + fn test_string_conversion() { + let cases = vec![ + (DataType::Utf8View, DataType::Utf8, true), + (DataType::Utf8View, DataType::LargeUtf8, true), + ]; + + for case in cases { + assert_eq!(can_coerce_from(&case.0, &case.1), case.2); + } + } + #[test] fn test_maybe_data_types() { // this vec contains: arg1, arg2, expected result diff --git a/datafusion/functions/src/string/starts_with.rs b/datafusion/functions/src/string/starts_with.rs index 05bd960ff14b..8450697cbf30 100644 --- a/datafusion/functions/src/string/starts_with.rs +++ b/datafusion/functions/src/string/starts_with.rs @@ -18,10 +18,10 @@ use std::any::Any; use std::sync::Arc; -use arrow::array::{ArrayRef, OffsetSizeTrait}; +use arrow::array::ArrayRef; use arrow::datatypes::DataType; -use datafusion_common::{cast::as_generic_string_array, internal_err, Result}; +use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; @@ -30,12 +30,8 @@ use crate::utils::make_scalar_function; /// Returns true if string starts with prefix. /// starts_with('alphabet', 'alph') = 't' -pub fn starts_with(args: &[ArrayRef]) -> Result { - let left = as_generic_string_array::(&args[0])?; - let right = as_generic_string_array::(&args[1])?; - - let result = arrow::compute::kernels::comparison::starts_with(left, right)?; - +pub fn starts_with(args: &[ArrayRef]) -> Result { + let result = arrow::compute::kernels::comparison::starts_with(&args[0], &args[1])?; Ok(Arc::new(result) as ArrayRef) } @@ -52,14 +48,15 @@ impl Default for StartsWithFunc { impl StartsWithFunc { pub fn new() -> Self { - use DataType::*; Self { signature: Signature::one_of( vec![ - Exact(vec![Utf8, Utf8]), - Exact(vec![Utf8, LargeUtf8]), - Exact(vec![LargeUtf8, Utf8]), - Exact(vec![LargeUtf8, LargeUtf8]), + // Planner attempts coercion to the target type starting with the most preferred candidate. + // For example, given input `(Utf8View, Utf8)`, it first tries coercing to `(Utf8View, Utf8View)`. + // If that fails, it proceeds to `(Utf8, Utf8)`. + Exact(vec![DataType::Utf8View, DataType::Utf8View]), + Exact(vec![DataType::Utf8, DataType::Utf8]), + Exact(vec![DataType::LargeUtf8, DataType::LargeUtf8]), ], Volatility::Immutable, ), @@ -81,18 +78,73 @@ impl ScalarUDFImpl for StartsWithFunc { } fn return_type(&self, _arg_types: &[DataType]) -> Result { - use DataType::*; - - Ok(Boolean) + Ok(DataType::Boolean) } fn invoke(&self, args: &[ColumnarValue]) -> Result { match args[0].data_type() { - DataType::Utf8 => make_scalar_function(starts_with::, vec![])(args), - DataType::LargeUtf8 => { - return make_scalar_function(starts_with::, vec![])(args); + DataType::Utf8View | DataType::Utf8 | DataType::LargeUtf8 => { + make_scalar_function(starts_with, vec![])(args) } - _ => internal_err!("Unsupported data type"), + _ => internal_err!("Unsupported data types for starts_with. Expected Utf8, LargeUtf8 or Utf8View")?, } } } + +#[cfg(test)] +mod tests { + use crate::utils::test::test_function; + use arrow::array::{Array, BooleanArray}; + use arrow::datatypes::DataType::Boolean; + use datafusion_common::{Result, ScalarValue}; + use datafusion_expr::{ColumnarValue, ScalarUDFImpl}; + + use super::*; + + #[test] + fn test_functions() -> Result<()> { + // Generate test cases for starts_with + let test_cases = vec![ + (Some("alphabet"), Some("alph"), Some(true)), + (Some("alphabet"), Some("bet"), Some(false)), + ( + Some("somewhat large string"), + Some("somewhat large"), + Some(true), + ), + (Some("somewhat large string"), Some("large"), Some(false)), + ] + .into_iter() + .flat_map(|(a, b, c)| { + let utf_8_args = vec![ + ColumnarValue::Scalar(ScalarValue::Utf8(a.map(|s| s.to_string()))), + ColumnarValue::Scalar(ScalarValue::Utf8(b.map(|s| s.to_string()))), + ]; + + let large_utf_8_args = vec![ + ColumnarValue::Scalar(ScalarValue::LargeUtf8(a.map(|s| s.to_string()))), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(b.map(|s| s.to_string()))), + ]; + + let utf_8_view_args = vec![ + ColumnarValue::Scalar(ScalarValue::Utf8View(a.map(|s| s.to_string()))), + ColumnarValue::Scalar(ScalarValue::Utf8View(b.map(|s| s.to_string()))), + ]; + + vec![(utf_8_args, c), (large_utf_8_args, c), (utf_8_view_args, c)] + }); + + for (args, expected) in test_cases { + test_function!( + StartsWithFunc::new(), + &args, + Ok(expected), + bool, + Boolean, + BooleanArray + ); + } + + Ok(()) + } +} diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index 763b4e99c614..584d3b330690 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -355,6 +355,75 @@ logical_plan 01)Aggregate: groupBy=[[]], aggr=[[count(DISTINCT test.column1_utf8), count(DISTINCT test.column1_utf8view), count(DISTINCT test.column1_dict)]] 02)--TableScan: test projection=[column1_utf8, column1_utf8view, column1_dict] +### `STARTS_WITH` + +# Test STARTS_WITH with utf8view against utf8view, utf8, and largeutf8 +# (should be no casts) +query TT +EXPLAIN SELECT + STARTS_WITH(column1_utf8view, column2_utf8view) as c1, + STARTS_WITH(column1_utf8view, column2_utf8) as c2, + STARTS_WITH(column1_utf8view, column2_large_utf8) as c3 +FROM test; +---- +logical_plan +01)Projection: starts_with(test.column1_utf8view, test.column2_utf8view) AS c1, starts_with(test.column1_utf8view, CAST(test.column2_utf8 AS Utf8View)) AS c2, starts_with(test.column1_utf8view, CAST(test.column2_large_utf8 AS Utf8View)) AS c3 +02)--TableScan: test projection=[column2_utf8, column2_large_utf8, column1_utf8view, column2_utf8view] + +query BBB +SELECT + STARTS_WITH(column1_utf8view, column2_utf8view) as c1, + STARTS_WITH(column1_utf8view, column2_utf8) as c2, + STARTS_WITH(column1_utf8view, column2_large_utf8) as c3 +FROM test; +---- +false false false +true true true +true true true +NULL NULL NULL + +# Test STARTS_WITH with utf8 against utf8view, utf8, and largeutf8 +# Should work, but will have to cast to common types +# should cast utf8 -> utf8view and largeutf8 -> utf8view +query TT +EXPLAIN SELECT + STARTS_WITH(column1_utf8, column2_utf8view) as c1, + STARTS_WITH(column1_utf8, column2_utf8) as c3, + STARTS_WITH(column1_utf8, column2_large_utf8) as c4 +FROM test; +---- +logical_plan +01)Projection: starts_with(__common_expr_1, test.column2_utf8view) AS c1, starts_with(test.column1_utf8, test.column2_utf8) AS c3, starts_with(__common_expr_1, CAST(test.column2_large_utf8 AS Utf8View)) AS c4 +02)--Projection: CAST(test.column1_utf8 AS Utf8View) AS __common_expr_1, test.column1_utf8, test.column2_utf8, test.column2_large_utf8, test.column2_utf8view +03)----TableScan: test projection=[column1_utf8, column2_utf8, column2_large_utf8, column2_utf8view] + +query BBB + SELECT + STARTS_WITH(column1_utf8, column2_utf8view) as c1, + STARTS_WITH(column1_utf8, column2_utf8) as c3, + STARTS_WITH(column1_utf8, column2_large_utf8) as c4 +FROM test; +---- +false false false +true true true +true true true +NULL NULL NULL + + +# Test STARTS_WITH with utf8view against literals +# In this case, the literals should be cast to utf8view. The columns +# should not be cast to utf8. +query TT +EXPLAIN SELECT + STARTS_WITH(column1_utf8view, 'äöüß') as c1, + STARTS_WITH(column1_utf8view, '') as c2, + STARTS_WITH(column1_utf8view, NULL) as c3, + STARTS_WITH(NULL, column1_utf8view) as c4 +FROM test; +---- +logical_plan +01)Projection: starts_with(test.column1_utf8view, Utf8View("äöüß")) AS c1, starts_with(test.column1_utf8view, Utf8View("")) AS c2, starts_with(test.column1_utf8view, Utf8View(NULL)) AS c3, starts_with(Utf8View(NULL), test.column1_utf8view) AS c4 +02)--TableScan: test projection=[column1_utf8view] statement ok drop table test; @@ -376,6 +445,5 @@ select t.dt from dates t where arrow_cast('2024-01-01', 'Utf8View') < t.dt; ---- 2024-01-23 - statement ok drop table dates; From 4e278cad19c350f93195e0953881fefafe442f55 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Tue, 6 Aug 2024 14:00:01 +0300 Subject: [PATCH 21/34] fix: hash join tests with forced collisions (#11806) * tests: hash join tests with hash collisions * replace div_ceil expression with function call * Apply suggestions from code review Co-authored-by: Andrew Lamb * remove redundant comments --------- Co-authored-by: Andrew Lamb --- .../physical-plan/src/joins/hash_join.rs | 83 ++++++++++++------- 1 file changed, 51 insertions(+), 32 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 063f35059fb8..14835f717ea3 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1583,7 +1583,6 @@ mod tests { use rstest::*; use rstest_reuse::*; - #[cfg(not(feature = "force_hash_collisions"))] fn div_ceil(a: usize, b: usize) -> usize { (a + b - 1) / b } @@ -1931,9 +1930,6 @@ mod tests { Ok(()) } - // FIXME(#TODO) test fails with feature `force_hash_collisions` - // https://github.com/apache/datafusion/issues/11658 - #[cfg(not(feature = "force_hash_collisions"))] #[apply(batch_sizes)] #[tokio::test] async fn join_inner_two(batch_size: usize) -> Result<()> { @@ -1964,12 +1960,20 @@ mod tests { assert_eq!(columns, vec!["a1", "b2", "c1", "a1", "b2", "c2"]); - // expected joined records = 3 - // in case batch_size is 1 - additional empty batch for remaining 3-2 row - let mut expected_batch_count = div_ceil(3, batch_size); - if batch_size == 1 { - expected_batch_count += 1; - } + let expected_batch_count = if cfg!(not(feature = "force_hash_collisions")) { + // Expected number of hash table matches = 3 + // in case batch_size is 1 - additional empty batch for remaining 3-2 row + let mut expected_batch_count = div_ceil(3, batch_size); + if batch_size == 1 { + expected_batch_count += 1; + } + expected_batch_count + } else { + // With hash collisions enabled, all records will match each other + // and filtered later. + div_ceil(9, batch_size) + }; + assert_eq!(batches.len(), expected_batch_count); let expected = [ @@ -1989,9 +1993,6 @@ mod tests { } /// Test where the left has 2 parts, the right with 1 part => 1 part - // FIXME(#TODO) test fails with feature `force_hash_collisions` - // https://github.com/apache/datafusion/issues/11658 - #[cfg(not(feature = "force_hash_collisions"))] #[apply(batch_sizes)] #[tokio::test] async fn join_inner_one_two_parts_left(batch_size: usize) -> Result<()> { @@ -2029,12 +2030,20 @@ mod tests { assert_eq!(columns, vec!["a1", "b2", "c1", "a1", "b2", "c2"]); - // expected joined records = 3 - // in case batch_size is 1 - additional empty batch for remaining 3-2 row - let mut expected_batch_count = div_ceil(3, batch_size); - if batch_size == 1 { - expected_batch_count += 1; - } + let expected_batch_count = if cfg!(not(feature = "force_hash_collisions")) { + // Expected number of hash table matches = 3 + // in case batch_size is 1 - additional empty batch for remaining 3-2 row + let mut expected_batch_count = div_ceil(3, batch_size); + if batch_size == 1 { + expected_batch_count += 1; + } + expected_batch_count + } else { + // With hash collisions enabled, all records will match each other + // and filtered later. + div_ceil(9, batch_size) + }; + assert_eq!(batches.len(), expected_batch_count); let expected = [ @@ -2104,9 +2113,6 @@ mod tests { } /// Test where the left has 1 part, the right has 2 parts => 2 parts - // FIXME(#TODO) test fails with feature `force_hash_collisions` - // https://github.com/apache/datafusion/issues/11658 - #[cfg(not(feature = "force_hash_collisions"))] #[apply(batch_sizes)] #[tokio::test] async fn join_inner_one_two_parts_right(batch_size: usize) -> Result<()> { @@ -2143,12 +2149,19 @@ mod tests { let stream = join.execute(0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; - // expected joined records = 1 (first right batch) - // and additional empty batch for non-joined 20-6-80 - let mut expected_batch_count = div_ceil(1, batch_size); - if batch_size == 1 { - expected_batch_count += 1; - } + let expected_batch_count = if cfg!(not(feature = "force_hash_collisions")) { + // Expected number of hash table matches for first right batch = 1 + // and additional empty batch for non-joined 20-6-80 + let mut expected_batch_count = div_ceil(1, batch_size); + if batch_size == 1 { + expected_batch_count += 1; + } + expected_batch_count + } else { + // With hash collisions enabled, all records will match each other + // and filtered later. + div_ceil(6, batch_size) + }; assert_eq!(batches.len(), expected_batch_count); let expected = [ @@ -2166,8 +2179,14 @@ mod tests { let stream = join.execute(1, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; - // expected joined records = 2 (second right batch) - let expected_batch_count = div_ceil(2, batch_size); + let expected_batch_count = if cfg!(not(feature = "force_hash_collisions")) { + // Expected number of hash table matches for second right batch = 2 + div_ceil(2, batch_size) + } else { + // With hash collisions enabled, all records will match each other + // and filtered later. + div_ceil(3, batch_size) + }; assert_eq!(batches.len(), expected_batch_count); let expected = [ @@ -3732,9 +3751,9 @@ mod tests { | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { - (expected_resultset_records + batch_size - 1) / batch_size + div_ceil(expected_resultset_records, batch_size) } - _ => (expected_resultset_records + batch_size - 1) / batch_size + 1, + _ => div_ceil(expected_resultset_records, batch_size) + 1, }; assert_eq!( batches.len(), From e19e9825f2e127621782f5fdcd5f22e04ace8ee8 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 6 Aug 2024 20:19:12 +0800 Subject: [PATCH 22/34] don't create zero sized buffer (#11841) --- datafusion/physical-plan/src/coalesce_batches.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index b822ec2dafeb..de42a55ad350 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -492,8 +492,10 @@ fn gc_string_view_batch(batch: &RecordBatch) -> RecordBatch { if actual_buffer_size > (ideal_buffer_size * 2) { // We set the block size to `ideal_buffer_size` so that the new StringViewArray only has one buffer, which accelerate later concat_batches. // See https://github.com/apache/arrow-rs/issues/6094 for more details. - let mut builder = StringViewBuilder::with_capacity(s.len()) - .with_block_size(ideal_buffer_size as u32); + let mut builder = StringViewBuilder::with_capacity(s.len()); + if ideal_buffer_size > 0 { + builder = builder.with_block_size(ideal_buffer_size as u32); + } for v in s.iter() { builder.append_option(v); @@ -802,7 +804,7 @@ mod tests { impl StringViewTest { /// Create a `StringViewArray` with the parameters specified in this struct fn build(self) -> StringViewArray { - let mut builder = StringViewBuilder::with_capacity(100); + let mut builder = StringViewBuilder::with_capacity(100).with_block_size(8192); loop { for &v in self.strings.iter() { builder.append_option(v); From 16a3557325eb8f949f4a87ab90c0a0b174dc8d86 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Tue, 6 Aug 2024 15:42:00 +0200 Subject: [PATCH 23/34] fix: `collect_columns` quadratic complexity (#11843) Fix accidental $O(n^2)$ in `collect_columns`. There are the following ways to insert a clone into a hash set: - **clone before check:** Basically `set.insert(x.clone())`. That's rather expensive if you have a lot of duplicates. - **iter & clone:** That's what we do right now, but that's in $O(n^2)$. - **check & insert:** Basically `if !set.contains(x) {set.insert(x.clone())}` That requires two hash probes though. - **entry-based API:** Sadly the stdlib doesn't really offer any handle/entry-based APIs yet (see https://github.com/rust-lang/rust/issues/60896 ), but `hashbrown` does, so we can use the nice `set.get_or_insert_owned(x)` which will only clone the reference if it doesn't exists yet and only hashes once. We now use the last approach. --- datafusion/physical-expr/src/utils/mod.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 6c4791b158c8..4c37db4849a7 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -17,9 +17,10 @@ mod guarantee; pub use guarantee::{Guarantee, LiteralGuarantee}; +use hashbrown::HashSet; use std::borrow::Borrow; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::sync::Arc; use crate::expressions::{BinaryExpr, Column}; @@ -204,9 +205,7 @@ pub fn collect_columns(expr: &Arc) -> HashSet { let mut columns = HashSet::::new(); expr.apply(|expr| { if let Some(column) = expr.as_any().downcast_ref::() { - if !columns.iter().any(|c| c.eq(column)) { - columns.insert(column.clone()); - } + columns.get_or_insert_owned(column); } Ok(TreeNodeRecursion::Continue) }) From bddb6415a50746d2803dd908d19c3758952d74f9 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 7 Aug 2024 03:17:36 +0800 Subject: [PATCH 24/34] Reduce clone of `Statistics` in `ListingTable` and `PartitionedFile` (#11802) * reduce clone of `Statistics` by using arc. * optimize `get_statistics_with_limit` and `split_files`. * directly create the col stats set. * fix pb. * fix fmt. * fix clippy. * fix compile. * remove stale codes. * optimize `split_files` by using drain. * remove default for PartitionedFile. * don't keep `Arc` in `PartitionedFile`. * fix pb. --- .../core/src/datasource/listing/helpers.rs | 21 ++- datafusion/core/src/datasource/listing/mod.rs | 1 + .../core/src/datasource/listing/table.rs | 26 +-- datafusion/core/src/datasource/statistics.rs | 156 +++++++++--------- 4 files changed, 107 insertions(+), 97 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 29b593a70ca0..67af8ef12c8b 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -18,6 +18,7 @@ //! Helper functions for the table implementation use std::collections::HashMap; +use std::mem; use std::sync::Arc; use super::PartitionedFile; @@ -138,10 +139,22 @@ pub fn split_files( // effectively this is div with rounding up instead of truncating let chunk_size = (partitioned_files.len() + n - 1) / n; - partitioned_files - .chunks(chunk_size) - .map(|c| c.to_vec()) - .collect() + let mut chunks = Vec::with_capacity(n); + let mut current_chunk = Vec::with_capacity(chunk_size); + for file in partitioned_files.drain(..) { + current_chunk.push(file); + if current_chunk.len() == chunk_size { + let full_chunk = + mem::replace(&mut current_chunk, Vec::with_capacity(chunk_size)); + chunks.push(full_chunk); + } + } + + if !current_chunk.is_empty() { + chunks.push(current_chunk) + } + + chunks } struct Partition { diff --git a/datafusion/core/src/datasource/listing/mod.rs b/datafusion/core/src/datasource/listing/mod.rs index 44f92760908d..21a60614cff2 100644 --- a/datafusion/core/src/datasource/listing/mod.rs +++ b/datafusion/core/src/datasource/listing/mod.rs @@ -82,6 +82,7 @@ pub struct PartitionedFile { /// An optional field for user defined per object metadata pub extensions: Option>, } + impl PartitionedFile { /// Create a simple file without metadata or partition pub fn new(path: impl Into, size: u64) -> Self { diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 80f49e4eb8e6..bb86ac3ae416 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -973,15 +973,16 @@ impl ListingTable { // collect the statistics if required by the config let files = file_list .map(|part_file| async { - let mut part_file = part_file?; + let part_file = part_file?; if self.options.collect_stat { let statistics = self.do_collect_statistics(ctx, &store, &part_file).await?; - part_file.statistics = Some(statistics.clone()); - Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> + Ok((part_file, statistics)) } else { - Ok((part_file, Statistics::new_unknown(&self.file_schema))) - as Result<(PartitionedFile, Statistics)> + Ok(( + part_file, + Arc::new(Statistics::new_unknown(&self.file_schema)), + )) } }) .boxed() @@ -1011,12 +1012,12 @@ impl ListingTable { ctx: &SessionState, store: &Arc, part_file: &PartitionedFile, - ) -> Result { - let statistics_cache = self.collected_statistics.clone(); - return match statistics_cache + ) -> Result> { + match self + .collected_statistics .get_with_extra(&part_file.object_meta.location, &part_file.object_meta) { - Some(statistics) => Ok(statistics.as_ref().clone()), + Some(statistics) => Ok(statistics.clone()), None => { let statistics = self .options @@ -1028,14 +1029,15 @@ impl ListingTable { &part_file.object_meta, ) .await?; - statistics_cache.put_with_extra( + let statistics = Arc::new(statistics); + self.collected_statistics.put_with_extra( &part_file.object_meta.location, - statistics.clone().into(), + statistics.clone(), &part_file.object_meta, ); Ok(statistics) } - }; + } } } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 8c789e461b08..9d031a6bbc85 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::mem; +use std::sync::Arc; + use super::listing::PartitionedFile; use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; @@ -26,8 +29,6 @@ use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; use futures::{Stream, StreamExt}; -use itertools::izip; -use itertools::multiunzip; /// Get all files as well as the file level summary statistics (no statistic for partition columns). /// If the optional `limit` is provided, includes only sufficient files. Needed to read up to @@ -35,7 +36,7 @@ use itertools::multiunzip; /// `ListingTable`. If it is false we only construct bare statistics and skip a potentially expensive /// call to `multiunzip` for constructing file level summary statistics. pub async fn get_statistics_with_limit( - all_files: impl Stream>, + all_files: impl Stream)>>, file_schema: SchemaRef, limit: Option, collect_stats: bool, @@ -48,9 +49,7 @@ pub async fn get_statistics_with_limit( // - zero for summations, and // - neutral element for extreme points. let size = file_schema.fields().len(); - let mut null_counts: Vec> = vec![Precision::Absent; size]; - let mut max_values: Vec> = vec![Precision::Absent; size]; - let mut min_values: Vec> = vec![Precision::Absent; size]; + let mut col_stats_set = vec![ColumnStatistics::default(); size]; let mut num_rows = Precision::::Absent; let mut total_byte_size = Precision::::Absent; @@ -58,16 +57,19 @@ pub async fn get_statistics_with_limit( let mut all_files = Box::pin(all_files.fuse()); if let Some(first_file) = all_files.next().await { - let (file, file_stats) = first_file?; + let (mut file, file_stats) = first_file?; + file.statistics = Some(file_stats.as_ref().clone()); result_files.push(file); // First file, we set them directly from the file statistics. - num_rows = file_stats.num_rows; - total_byte_size = file_stats.total_byte_size; - for (index, file_column) in file_stats.column_statistics.into_iter().enumerate() { - null_counts[index] = file_column.null_count; - max_values[index] = file_column.max_value; - min_values[index] = file_column.min_value; + num_rows = file_stats.num_rows.clone(); + total_byte_size = file_stats.total_byte_size.clone(); + for (index, file_column) in + file_stats.column_statistics.clone().into_iter().enumerate() + { + col_stats_set[index].null_count = file_column.null_count; + col_stats_set[index].max_value = file_column.max_value; + col_stats_set[index].min_value = file_column.min_value; } // If the number of rows exceeds the limit, we can stop processing @@ -80,7 +82,8 @@ pub async fn get_statistics_with_limit( }; if conservative_num_rows <= limit.unwrap_or(usize::MAX) { while let Some(current) = all_files.next().await { - let (file, file_stats) = current?; + let (mut file, file_stats) = current?; + file.statistics = Some(file_stats.as_ref().clone()); result_files.push(file); if !collect_stats { continue; @@ -90,38 +93,28 @@ pub async fn get_statistics_with_limit( // counts across all the files in question. If any file does not // provide any information or provides an inexact value, we demote // the statistic precision to inexact. - num_rows = add_row_stats(file_stats.num_rows, num_rows); + num_rows = add_row_stats(file_stats.num_rows.clone(), num_rows); total_byte_size = - add_row_stats(file_stats.total_byte_size, total_byte_size); + add_row_stats(file_stats.total_byte_size.clone(), total_byte_size); - (null_counts, max_values, min_values) = multiunzip( - izip!( - file_stats.column_statistics.into_iter(), - null_counts.into_iter(), - max_values.into_iter(), - min_values.into_iter() - ) - .map( - |( - ColumnStatistics { - null_count: file_nc, - max_value: file_max, - min_value: file_min, - distinct_count: _, - }, - null_count, - max_value, - min_value, - )| { - ( - add_row_stats(file_nc, null_count), - set_max_if_greater(file_max, max_value), - set_min_if_lesser(file_min, min_value), - ) - }, - ), - ); + for (file_col_stats, col_stats) in file_stats + .column_statistics + .iter() + .zip(col_stats_set.iter_mut()) + { + let ColumnStatistics { + null_count: file_nc, + max_value: file_max, + min_value: file_min, + distinct_count: _, + } = file_col_stats; + + col_stats.null_count = + add_row_stats(file_nc.clone(), col_stats.null_count.clone()); + set_max_if_greater(file_max, &mut col_stats.max_value); + set_min_if_lesser(file_min, &mut col_stats.min_value) + } // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also @@ -139,7 +132,7 @@ pub async fn get_statistics_with_limit( let mut statistics = Statistics { num_rows, total_byte_size, - column_statistics: get_col_stats_vec(null_counts, max_values, min_values), + column_statistics: col_stats_set, }; if all_files.next().await.is_some() { // If we still have files in the stream, it means that the limit kicked @@ -182,21 +175,6 @@ fn add_row_stats( } } -pub(crate) fn get_col_stats_vec( - null_counts: Vec>, - max_values: Vec>, - min_values: Vec>, -) -> Vec { - izip!(null_counts, max_values, min_values) - .map(|(null_count, max_value, min_value)| ColumnStatistics { - null_count, - max_value, - min_value, - distinct_count: Precision::Absent, - }) - .collect() -} - pub(crate) fn get_col_stats( schema: &Schema, null_counts: Vec>, @@ -238,45 +216,61 @@ fn min_max_aggregate_data_type(input_type: &DataType) -> &DataType { /// If the given value is numerically greater than the original maximum value, /// return the new maximum value with appropriate exactness information. fn set_max_if_greater( - max_nominee: Precision, - max_values: Precision, -) -> Precision { - match (&max_values, &max_nominee) { - (Precision::Exact(val1), Precision::Exact(val2)) if val1 < val2 => max_nominee, + max_nominee: &Precision, + max_value: &mut Precision, +) { + match (&max_value, max_nominee) { + (Precision::Exact(val1), Precision::Exact(val2)) if val1 < val2 => { + *max_value = max_nominee.clone(); + } (Precision::Exact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Exact(val2)) if val1 < val2 => { - max_nominee.to_inexact() + *max_value = max_nominee.clone().to_inexact(); + } + (Precision::Exact(_), Precision::Absent) => { + let exact_max = mem::take(max_value); + *max_value = exact_max.to_inexact(); + } + (Precision::Absent, Precision::Exact(_)) => { + *max_value = max_nominee.clone().to_inexact(); + } + (Precision::Absent, Precision::Inexact(_)) => { + *max_value = max_nominee.clone(); } - (Precision::Exact(_), Precision::Absent) => max_values.to_inexact(), - (Precision::Absent, Precision::Exact(_)) => max_nominee.to_inexact(), - (Precision::Absent, Precision::Inexact(_)) => max_nominee, - (Precision::Absent, Precision::Absent) => Precision::Absent, - _ => max_values, + _ => {} } } /// If the given value is numerically lesser than the original minimum value, /// return the new minimum value with appropriate exactness information. fn set_min_if_lesser( - min_nominee: Precision, - min_values: Precision, -) -> Precision { - match (&min_values, &min_nominee) { - (Precision::Exact(val1), Precision::Exact(val2)) if val1 > val2 => min_nominee, + min_nominee: &Precision, + min_value: &mut Precision, +) { + match (&min_value, min_nominee) { + (Precision::Exact(val1), Precision::Exact(val2)) if val1 > val2 => { + *min_value = min_nominee.clone(); + } (Precision::Exact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Exact(val2)) if val1 > val2 => { - min_nominee.to_inexact() + *min_value = min_nominee.clone().to_inexact(); + } + (Precision::Exact(_), Precision::Absent) => { + let exact_min = mem::take(min_value); + *min_value = exact_min.to_inexact(); + } + (Precision::Absent, Precision::Exact(_)) => { + *min_value = min_nominee.clone().to_inexact(); + } + (Precision::Absent, Precision::Inexact(_)) => { + *min_value = min_nominee.clone(); } - (Precision::Exact(_), Precision::Absent) => min_values.to_inexact(), - (Precision::Absent, Precision::Exact(_)) => min_nominee.to_inexact(), - (Precision::Absent, Precision::Inexact(_)) => min_nominee, - (Precision::Absent, Precision::Absent) => Precision::Absent, - _ => min_values, + _ => {} } } From 117ab1b8c861f431b2d04443e2028ca16fafc57c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 7 Aug 2024 16:39:24 +0800 Subject: [PATCH 25/34] Add `LogicalPlan::CreateIndex` (#11817) * Add create index plan * Fix clippy lints --- datafusion/core/src/execution/context/mod.rs | 29 ++++++---- datafusion/expr/src/logical_plan/ddl.rs | 19 ++++++ datafusion/expr/src/logical_plan/mod.rs | 4 +- datafusion/expr/src/logical_plan/tree_node.rs | 1 + datafusion/proto/src/logical_plan/mod.rs | 3 + datafusion/sql/src/statement.rs | 58 +++++++++++++++---- datafusion/sql/tests/sql_integration.rs | 34 ++++++++++- 7 files changed, 121 insertions(+), 27 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index e6bb1483e256..c883b7a47fbb 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -544,30 +544,35 @@ impl SessionContext { // stack overflows. match ddl { DdlStatement::CreateExternalTable(cmd) => { - Box::pin(async move { self.create_external_table(&cmd).await }) - as std::pin::Pin + Send>> + (Box::pin(async move { self.create_external_table(&cmd).await }) + as std::pin::Pin + Send>>) + .await } DdlStatement::CreateMemoryTable(cmd) => { - Box::pin(self.create_memory_table(cmd)) + Box::pin(self.create_memory_table(cmd)).await + } + DdlStatement::CreateView(cmd) => { + Box::pin(self.create_view(cmd)).await } - DdlStatement::CreateView(cmd) => Box::pin(self.create_view(cmd)), DdlStatement::CreateCatalogSchema(cmd) => { - Box::pin(self.create_catalog_schema(cmd)) + Box::pin(self.create_catalog_schema(cmd)).await } DdlStatement::CreateCatalog(cmd) => { - Box::pin(self.create_catalog(cmd)) + Box::pin(self.create_catalog(cmd)).await } - DdlStatement::DropTable(cmd) => Box::pin(self.drop_table(cmd)), - DdlStatement::DropView(cmd) => Box::pin(self.drop_view(cmd)), + DdlStatement::DropTable(cmd) => Box::pin(self.drop_table(cmd)).await, + DdlStatement::DropView(cmd) => Box::pin(self.drop_view(cmd)).await, DdlStatement::DropCatalogSchema(cmd) => { - Box::pin(self.drop_schema(cmd)) + Box::pin(self.drop_schema(cmd)).await } DdlStatement::CreateFunction(cmd) => { - Box::pin(self.create_function(cmd)) + Box::pin(self.create_function(cmd)).await + } + DdlStatement::DropFunction(cmd) => { + Box::pin(self.drop_function(cmd)).await } - DdlStatement::DropFunction(cmd) => Box::pin(self.drop_function(cmd)), + ddl => Ok(DataFrame::new(self.state(), LogicalPlan::Ddl(ddl))), } - .await } // TODO what about the other statements (like TransactionStart and TransactionEnd) LogicalPlan::Statement(Statement::SetVariable(stmt)) => { diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 255bf4699b7f..ad0fcd2d4771 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -41,6 +41,8 @@ pub enum DdlStatement { CreateCatalogSchema(CreateCatalogSchema), /// Creates a new catalog (aka "Database"). CreateCatalog(CreateCatalog), + /// Creates a new index. + CreateIndex(CreateIndex), /// Drops a table. DropTable(DropTable), /// Drops a view. @@ -66,6 +68,7 @@ impl DdlStatement { schema } DdlStatement::CreateCatalog(CreateCatalog { schema, .. }) => schema, + DdlStatement::CreateIndex(CreateIndex { schema, .. }) => schema, DdlStatement::DropTable(DropTable { schema, .. }) => schema, DdlStatement::DropView(DropView { schema, .. }) => schema, DdlStatement::DropCatalogSchema(DropCatalogSchema { schema, .. }) => schema, @@ -83,6 +86,7 @@ impl DdlStatement { DdlStatement::CreateView(_) => "CreateView", DdlStatement::CreateCatalogSchema(_) => "CreateCatalogSchema", DdlStatement::CreateCatalog(_) => "CreateCatalog", + DdlStatement::CreateIndex(_) => "CreateIndex", DdlStatement::DropTable(_) => "DropTable", DdlStatement::DropView(_) => "DropView", DdlStatement::DropCatalogSchema(_) => "DropCatalogSchema", @@ -101,6 +105,7 @@ impl DdlStatement { vec![input] } DdlStatement::CreateView(CreateView { input, .. }) => vec![input], + DdlStatement::CreateIndex(_) => vec![], DdlStatement::DropTable(_) => vec![], DdlStatement::DropView(_) => vec![], DdlStatement::DropCatalogSchema(_) => vec![], @@ -147,6 +152,9 @@ impl DdlStatement { }) => { write!(f, "CreateCatalog: {catalog_name:?}") } + DdlStatement::CreateIndex(CreateIndex { name, .. }) => { + write!(f, "CreateIndex: {name:?}") + } DdlStatement::DropTable(DropTable { name, if_exists, .. }) => { @@ -351,3 +359,14 @@ pub struct DropFunction { pub if_exists: bool, pub schema: DFSchemaRef, } + +#[derive(Clone, PartialEq, Eq, Hash, Debug)] +pub struct CreateIndex { + pub name: Option, + pub table: TableReference, + pub using: Option, + pub columns: Vec, + pub unique: bool, + pub if_not_exists: bool, + pub schema: DFSchemaRef, +} diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index 8928f70cd5d2..b58208591920 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -30,8 +30,8 @@ pub use builder::{ }; pub use ddl::{ CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateFunction, - CreateFunctionBody, CreateMemoryTable, CreateView, DdlStatement, DropCatalogSchema, - DropFunction, DropTable, DropView, OperateFunctionArg, + CreateFunctionBody, CreateIndex, CreateMemoryTable, CreateView, DdlStatement, + DropCatalogSchema, DropFunction, DropTable, DropView, OperateFunctionArg, }; pub use dml::{DmlStatement, WriteOp}; pub use plan::{ diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index a47906f20322..dbe43128fd38 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -303,6 +303,7 @@ impl TreeNode for LogicalPlan { DdlStatement::CreateExternalTable(_) | DdlStatement::CreateCatalogSchema(_) | DdlStatement::CreateCatalog(_) + | DdlStatement::CreateIndex(_) | DdlStatement::DropTable(_) | DdlStatement::DropView(_) | DdlStatement::DropCatalogSchema(_) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 0a91babdfb60..bc019725f36c 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -1645,6 +1645,9 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlan::Ddl(DdlStatement::CreateMemoryTable(_)) => Err(proto_error( "LogicalPlan serde is not yet implemented for CreateMemoryTable", )), + LogicalPlan::Ddl(DdlStatement::CreateIndex(_)) => Err(proto_error( + "LogicalPlan serde is not yet implemented for CreateIndex", + )), LogicalPlan::Ddl(DdlStatement::DropTable(_)) => Err(proto_error( "LogicalPlan serde is not yet implemented for DropTable", )), diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 3737e1adf8f3..6d47232ec270 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -45,20 +45,20 @@ use datafusion_expr::utils::expr_to_columns; use datafusion_expr::{ cast, col, Analyze, CreateCatalog, CreateCatalogSchema, CreateExternalTable as PlanCreateExternalTable, CreateFunction, CreateFunctionBody, - CreateMemoryTable, CreateView, DescribeTable, DmlStatement, DropCatalogSchema, - DropFunction, DropTable, DropView, EmptyRelation, Explain, Expr, ExprSchemable, - Filter, LogicalPlan, LogicalPlanBuilder, OperateFunctionArg, PlanType, Prepare, - SetVariable, Statement as PlanStatement, ToStringifiedPlan, TransactionAccessMode, - TransactionConclusion, TransactionEnd, TransactionIsolationLevel, TransactionStart, - Volatility, WriteOp, + CreateIndex as PlanCreateIndex, CreateMemoryTable, CreateView, DescribeTable, + DmlStatement, DropCatalogSchema, DropFunction, DropTable, DropView, EmptyRelation, + Explain, Expr, ExprSchemable, Filter, LogicalPlan, LogicalPlanBuilder, + OperateFunctionArg, PlanType, Prepare, SetVariable, Statement as PlanStatement, + ToStringifiedPlan, TransactionAccessMode, TransactionConclusion, TransactionEnd, + TransactionIsolationLevel, TransactionStart, Volatility, WriteOp, }; use sqlparser::ast; use sqlparser::ast::{ - Assignment, AssignmentTarget, ColumnDef, CreateTable, CreateTableOptions, Delete, - DescribeAlias, Expr as SQLExpr, FromTable, Ident, Insert, ObjectName, ObjectType, - OneOrManyWithParens, Query, SchemaName, SetExpr, ShowCreateObject, - ShowStatementFilter, Statement, TableConstraint, TableFactor, TableWithJoins, - TransactionMode, UnaryOperator, Value, + Assignment, AssignmentTarget, ColumnDef, CreateIndex, CreateTable, + CreateTableOptions, Delete, DescribeAlias, Expr as SQLExpr, FromTable, Ident, Insert, + ObjectName, ObjectType, OneOrManyWithParens, Query, SchemaName, SetExpr, + ShowCreateObject, ShowStatementFilter, Statement, TableConstraint, TableFactor, + TableWithJoins, TransactionMode, UnaryOperator, Value, }; use sqlparser::parser::ParserError::ParserError; @@ -769,6 +769,42 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { exec_err!("Function name not provided") } } + Statement::CreateIndex(CreateIndex { + name, + table_name, + using, + columns, + unique, + if_not_exists, + .. + }) => { + let name: Option = name.as_ref().map(object_name_to_string); + let table = self.object_name_to_table_reference(table_name)?; + let table_schema = self + .context_provider + .get_table_source(table.clone())? + .schema() + .to_dfschema_ref()?; + let using: Option = using.as_ref().map(ident_to_string); + let columns = self.order_by_to_sort_expr( + columns, + &table_schema, + planner_context, + false, + None, + )?; + Ok(LogicalPlan::Ddl(DdlStatement::CreateIndex( + PlanCreateIndex { + name, + table, + using, + columns, + unique, + if_not_exists, + schema: DFSchemaRef::new(DFSchema::empty()), + }, + ))) + } _ => { not_impl_err!("Unsupported SQL statement: {sql:?}") } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 8a5510eb69f3..4d7e60805657 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -28,11 +28,12 @@ use datafusion_common::{ assert_contains, DataFusionError, ParamValues, Result, ScalarValue, }; use datafusion_expr::{ + col, dml::CopyTo, logical_plan::{LogicalPlan, Prepare}, test::function_stub::sum_udaf, - ColumnarValue, CreateExternalTable, DdlStatement, ScalarUDF, ScalarUDFImpl, - Signature, Volatility, + ColumnarValue, CreateExternalTable, CreateIndex, DdlStatement, ScalarUDF, + ScalarUDFImpl, Signature, Volatility, }; use datafusion_functions::{string, unicode}; use datafusion_sql::{ @@ -4426,6 +4427,35 @@ fn test_parse_escaped_string_literal_value() { ) } +#[test] +fn plan_create_index() { + let sql = + "CREATE UNIQUE INDEX IF NOT EXISTS idx_name ON test USING btree (name, age DESC)"; + let plan = logical_plan_with_options(sql, ParserOptions::default()).unwrap(); + match plan { + LogicalPlan::Ddl(DdlStatement::CreateIndex(CreateIndex { + name, + table, + using, + columns, + unique, + if_not_exists, + .. + })) => { + assert_eq!(name, Some("idx_name".to_string())); + assert_eq!(format!("{table}"), "test"); + assert_eq!(using, Some("btree".to_string())); + assert_eq!( + columns, + vec![col("name").sort(true, false), col("age").sort(false, true),] + ); + assert!(unique); + assert!(if_not_exists); + } + _ => panic!("wrong plan type"), + } +} + fn assert_field_not_found(err: DataFusionError, name: &str) { match err { DataFusionError::SchemaError { .. } => { From 1ecdf900786844f09533e91f7e8fd52170ae6ae6 Mon Sep 17 00:00:00 2001 From: danlgrca Date: Wed, 7 Aug 2024 07:14:35 -0400 Subject: [PATCH 26/34] Update `object_store` to 0.10.2 (#11860) As part of mitigating https://nvd.nist.gov/vuln/detail/CVE-2024-41178, `object_store` should be upgraded to the no-longer-vulnerable version 0.10.2 instead of 0.10.1 --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 38b5627bc187..e1bd0d7aa72b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -116,7 +116,7 @@ indexmap = "2.0.0" itertools = "0.12" log = "^0.4" num_cpus = "1.13.0" -object_store = { version = "0.10.1", default-features = false } +object_store = { version = "0.10.2", default-features = false } parking_lot = "0.12" parquet = { version = "52.2.0", default-features = false, features = [ "arrow", From 679a85f808ce130ff354c762d29315eb2dc32b3f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 7 Aug 2024 12:59:06 -0400 Subject: [PATCH 27/34] Add metrics for skipped rows (#11706) --- .../physical-plan/src/aggregates/row_hash.rs | 47 ++++++++++++++----- 1 file changed, 34 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 1b84befb0269..ed3d6d49f9f3 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -28,12 +28,12 @@ use crate::aggregates::{ PhysicalGroupBy, }; use crate::common::IPCWriter; -use crate::metrics::{BaselineMetrics, RecordOutput}; +use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput}; use crate::sorts::sort::sort_batch; use crate::sorts::streaming_merge; use crate::spill::read_spill_as_stream; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, ExecutionPlan, PhysicalExpr}; +use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; @@ -117,10 +117,22 @@ struct SkipAggregationProbe { /// Flag indicating that further updates of `SkipAggregationProbe` /// state won't make any effect is_locked: bool, + + /// Number of rows where state was output without aggregation. + /// + /// * If 0, all input rows were aggregated (should_skip was always false) + /// + /// * if greater than zero, the number of rows which were output directly + /// without aggregation + skipped_aggregation_rows: metrics::Count, } impl SkipAggregationProbe { - fn new(probe_rows_threshold: usize, probe_ratio_threshold: f64) -> Self { + fn new( + probe_rows_threshold: usize, + probe_ratio_threshold: f64, + skipped_aggregation_rows: metrics::Count, + ) -> Self { Self { input_rows: 0, num_groups: 0, @@ -128,6 +140,7 @@ impl SkipAggregationProbe { probe_ratio_threshold, should_skip: false, is_locked: false, + skipped_aggregation_rows, } } @@ -160,6 +173,11 @@ impl SkipAggregationProbe { self.should_skip = false; self.is_locked = true; } + + /// Record the number of rows that were output directly without aggregation + fn record_skipped(&mut self, batch: &RecordBatch) { + self.skipped_aggregation_rows.add(batch.num_rows()); + } } /// HashTable based Grouping Aggregator @@ -473,17 +491,17 @@ impl GroupedHashAggregateStream { .all(|acc| acc.supports_convert_to_state()) && agg_group_by.is_single() { + let options = &context.session_config().options().execution; + let probe_rows_threshold = + options.skip_partial_aggregation_probe_rows_threshold; + let probe_ratio_threshold = + options.skip_partial_aggregation_probe_ratio_threshold; + let skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) + .counter("skipped_aggregation_rows", partition); Some(SkipAggregationProbe::new( - context - .session_config() - .options() - .execution - .skip_partial_aggregation_probe_rows_threshold, - context - .session_config() - .options() - .execution - .skip_partial_aggregation_probe_ratio_threshold, + probe_rows_threshold, + probe_ratio_threshold, + skipped_aggregation_rows, )) } else { None @@ -611,6 +629,9 @@ impl Stream for GroupedHashAggregateStream { match ready!(self.input.poll_next_unpin(cx)) { Some(Ok(batch)) => { let _timer = elapsed_compute.timer(); + if let Some(probe) = self.skip_aggregation_probe.as_mut() { + probe.record_skipped(&batch); + } let states = self.transform_to_states(batch)?; return Poll::Ready(Some(Ok( states.record_output(&self.baseline_metrics) From cd322f11455c2a95634268bb6064d1e888abdfec Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 7 Aug 2024 13:32:55 -0600 Subject: [PATCH 28/34] feat: Expose public method for optimizing physical plans (#11879) * expose public method for optimizing physical plans using the default planner * cargo fmt --- datafusion-examples/examples/planner_api.rs | 16 ++++++++++++++++ datafusion/core/src/physical_planner.rs | 6 +++--- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 92b58bcee197..35cf766ba1af 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -17,6 +17,7 @@ use datafusion::error::Result; use datafusion::physical_plan::displayable; +use datafusion::physical_planner::DefaultPhysicalPlanner; use datafusion::prelude::*; use datafusion_expr::{LogicalPlan, PlanType}; @@ -123,5 +124,20 @@ async fn to_physical_plan_step_by_step_demo( .plan ); + // Call the physical optimizer with an existing physical plan (in this + // case the plan is already optimized, but an unoptimized plan would + // typically be used in this context) + // Note that this is not part of the trait but a public method + // on DefaultPhysicalPlanner. Not all planners will provide this feature. + let planner = DefaultPhysicalPlanner::default(); + let physical_plan = + planner.optimize_physical_plan(physical_plan, &ctx.state(), |_, _| {})?; + println!( + "Optimized physical plan:\n\n{}\n\n", + displayable(physical_plan.as_ref()) + .to_stringified(false, PlanType::InitialPhysicalPlan) + .plan + ); + Ok(()) } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 65cdbf9fe62c..58b02c08e34c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -180,7 +180,7 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; - self.optimize_internal(plan, session_state, |_, _| {}) + self.optimize_physical_plan(plan, session_state, |_, _| {}) } } } @@ -1732,7 +1732,7 @@ impl DefaultPhysicalPlanner { } } - let optimized_plan = self.optimize_internal( + let optimized_plan = self.optimize_physical_plan( input, session_state, |plan, optimizer| { @@ -1816,7 +1816,7 @@ impl DefaultPhysicalPlanner { /// Optimize a physical plan by applying each physical optimizer, /// calling observer(plan, optimizer after each one) - fn optimize_internal( + pub fn optimize_physical_plan( &self, plan: Arc, session_state: &SessionState, From 60d1d3a702be9c95f14087337c693cd678d01dea Mon Sep 17 00:00:00 2001 From: Dharan Aditya Date: Thu, 8 Aug 2024 03:35:57 +0530 Subject: [PATCH 29/34] Cast `Utf8View` to `Utf8` to support `||` from `StringViewArray` (#11796) * make query work * hack string_concat_coercion * more tests --- datafusion/expr/src/type_coercion/binary.rs | 23 +++++--- .../sqllogictest/test_files/string_view.slt | 53 +++++++++++++++++++ 2 files changed, 68 insertions(+), 8 deletions(-) diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 17280289ed1b..8da33081d652 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -890,15 +890,22 @@ fn dictionary_coercion( /// 2. Data type of the other side should be able to cast to string type fn string_concat_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { use arrow::datatypes::DataType::*; - string_coercion(lhs_type, rhs_type).or(match (lhs_type, rhs_type) { - (Utf8, from_type) | (from_type, Utf8) => { - string_concat_internal_coercion(from_type, &Utf8) - } - (LargeUtf8, from_type) | (from_type, LargeUtf8) => { - string_concat_internal_coercion(from_type, &LargeUtf8) + match (lhs_type, rhs_type) { + // If Utf8View is in any side, we coerce to Utf8. + // Ref: https://github.com/apache/datafusion/pull/11796 + (Utf8View, Utf8View | Utf8 | LargeUtf8) | (Utf8 | LargeUtf8, Utf8View) => { + Some(Utf8) } - _ => None, - }) + _ => string_coercion(lhs_type, rhs_type).or(match (lhs_type, rhs_type) { + (Utf8, from_type) | (from_type, Utf8) => { + string_concat_internal_coercion(from_type, &Utf8) + } + (LargeUtf8, from_type) | (from_type, LargeUtf8) => { + string_concat_internal_coercion(from_type, &LargeUtf8) + } + _ => None, + }), + } } fn array_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index 584d3b330690..4d3f72b1e8d4 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -447,3 +447,56 @@ select t.dt from dates t where arrow_cast('2024-01-01', 'Utf8View') < t.dt; statement ok drop table dates; + +statement ok +create table temp as values +('value1', arrow_cast('rust', 'Utf8View'), arrow_cast('fast', 'Utf8View')), +('value2', arrow_cast('datafusion', 'Utf8View'), arrow_cast('cool', 'Utf8View')); + +query T +select column2||' is fast' from temp; +---- +rust is fast +datafusion is fast + + +query T +select column2 || ' is ' || column3 from temp; +---- +rust is fast +datafusion is cool + +query TT +explain select column2 || 'is' || column3 from temp; +---- +logical_plan +01)Projection: CAST(temp.column2 AS Utf8) || Utf8("is") || CAST(temp.column3 AS Utf8) +02)--TableScan: temp projection=[column2, column3] + + +query TT +explain select column2||' is fast' from temp; +---- +logical_plan +01)Projection: CAST(temp.column2 AS Utf8) || Utf8(" is fast") +02)--TableScan: temp projection=[column2] + + +query T +select column2||column3 from temp; +---- +rustfast +datafusioncool + +query TT +explain select column2||column3 from temp; +---- +logical_plan +01)Projection: CAST(temp.column2 AS Utf8) || CAST(temp.column3 AS Utf8) +02)--TableScan: temp projection=[column2, column3] + +query T +select column2|| ' ' ||column3 from temp; +---- +rust fast +datafusion cool From d0a1d3030a70eaba00eb856ce864c6a1cee02365 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Thu, 8 Aug 2024 09:45:43 +0800 Subject: [PATCH 30/34] refactor: remove unneed mut for session context (#11864) * doc: remove mut from session context docstring * refactor: remove unnecessary mut for session context * refactor: remove more unused mut --- .../examples/cli-session-context.rs | 6 ++--- datafusion-cli/src/catalog.rs | 2 +- datafusion-cli/src/command.rs | 2 +- datafusion-cli/src/exec.rs | 16 ++++++------ datafusion-cli/src/main.rs | 10 +++---- datafusion-examples/examples/catalog.rs | 2 +- datafusion/core/benches/filter_query_sql.rs | 14 +++++----- datafusion/core/src/dataframe/mod.rs | 14 +++++----- datafusion/core/src/dataframe/parquet.rs | 10 +++---- datafusion/core/src/execution/context/mod.rs | 13 ++++++---- datafusion/core/src/test_util/mod.rs | 6 ++--- .../tests/user_defined/user_defined_plan.rs | 26 +++++++++---------- .../tests/cases/roundtrip_logical_plan.rs | 4 ++- .../custom-table-providers.md | 2 +- 14 files changed, 65 insertions(+), 62 deletions(-) diff --git a/datafusion-cli/examples/cli-session-context.rs b/datafusion-cli/examples/cli-session-context.rs index 8da52ed84a5f..1a8f15c8731b 100644 --- a/datafusion-cli/examples/cli-session-context.rs +++ b/datafusion-cli/examples/cli-session-context.rs @@ -82,7 +82,7 @@ impl CliSessionContext for MyUnionerContext { #[tokio::main] /// Runs the example. pub async fn main() { - let mut my_ctx = MyUnionerContext::default(); + let my_ctx = MyUnionerContext::default(); let mut print_options = PrintOptions { format: datafusion_cli::print_format::PrintFormat::Automatic, @@ -91,7 +91,5 @@ pub async fn main() { color: true, }; - exec_from_repl(&mut my_ctx, &mut print_options) - .await - .unwrap(); + exec_from_repl(&my_ctx, &mut print_options).await.unwrap(); } diff --git a/datafusion-cli/src/catalog.rs b/datafusion-cli/src/catalog.rs index 273eb30d3a71..c4636f1ce0e0 100644 --- a/datafusion-cli/src/catalog.rs +++ b/datafusion-cli/src/catalog.rs @@ -240,7 +240,7 @@ mod tests { use datafusion::prelude::SessionContext; fn setup_context() -> (SessionContext, Arc) { - let mut ctx = SessionContext::new(); + let ctx = SessionContext::new(); ctx.register_catalog_list(Arc::new(DynamicFileCatalog::new( ctx.state().catalog_list().clone(), ctx.state_weak_ref(), diff --git a/datafusion-cli/src/command.rs b/datafusion-cli/src/command.rs index 1a6c023d3b50..05c00d634c94 100644 --- a/datafusion-cli/src/command.rs +++ b/datafusion-cli/src/command.rs @@ -55,7 +55,7 @@ pub enum OutputFormat { impl Command { pub async fn execute( &self, - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, print_options: &mut PrintOptions, ) -> Result<()> { match self { diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index b78f32e0ac48..178bce6f2fe6 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -49,7 +49,7 @@ use tokio::signal; /// run and execute SQL statements and commands, against a context with the given print options pub async fn exec_from_commands( - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, commands: Vec, print_options: &PrintOptions, ) -> Result<()> { @@ -62,7 +62,7 @@ pub async fn exec_from_commands( /// run and execute SQL statements and commands from a file, against a context with the given print options pub async fn exec_from_lines( - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, reader: &mut BufReader, print_options: &PrintOptions, ) -> Result<()> { @@ -102,7 +102,7 @@ pub async fn exec_from_lines( } pub async fn exec_from_files( - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, files: Vec, print_options: &PrintOptions, ) -> Result<()> { @@ -121,7 +121,7 @@ pub async fn exec_from_files( /// run and execute SQL statements and commands against a context with the given print options pub async fn exec_from_repl( - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, print_options: &mut PrintOptions, ) -> rustyline::Result<()> { let mut rl = Editor::new()?; @@ -204,7 +204,7 @@ pub async fn exec_from_repl( } pub(super) async fn exec_and_print( - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, print_options: &PrintOptions, sql: String, ) -> Result<()> { @@ -300,7 +300,7 @@ fn config_file_type_from_str(ext: &str) -> Option { } async fn create_plan( - ctx: &mut dyn CliSessionContext, + ctx: &dyn CliSessionContext, statement: Statement, ) -> Result { let mut plan = ctx.session_state().statement_to_plan(statement).await?; @@ -473,7 +473,7 @@ mod tests { "cos://bucket/path/file.parquet", "gcs://bucket/path/file.parquet", ]; - let mut ctx = SessionContext::new(); + let ctx = SessionContext::new(); let task_ctx = ctx.task_ctx(); let dialect = &task_ctx.session_config().options().sql_parser.dialect; let dialect = dialect_from_str(dialect).ok_or_else(|| { @@ -488,7 +488,7 @@ mod tests { let statements = DFParser::parse_sql_with_dialect(&sql, dialect.as_ref())?; for statement in statements { //Should not fail - let mut plan = create_plan(&mut ctx, statement).await?; + let mut plan = create_plan(&ctx, statement).await?; if let LogicalPlan::Copy(copy_to) = &mut plan { assert_eq!(copy_to.output_url, location); assert_eq!(copy_to.file_type.get_ext(), "parquet".to_string()); diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 6266ae6f561a..1810d3cef57c 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -175,7 +175,7 @@ async fn main_inner() -> Result<()> { let runtime_env = create_runtime_env(rt_config.clone())?; - let mut ctx = + let ctx = SessionContext::new_with_config_rt(session_config.clone(), Arc::new(runtime_env)); ctx.refresh_catalogs().await?; // install dynamic catalog provider that knows how to open files @@ -212,20 +212,20 @@ async fn main_inner() -> Result<()> { if commands.is_empty() && files.is_empty() { if !rc.is_empty() { - exec::exec_from_files(&mut ctx, rc, &print_options).await?; + exec::exec_from_files(&ctx, rc, &print_options).await?; } // TODO maybe we can have thiserror for cli but for now let's keep it simple - return exec::exec_from_repl(&mut ctx, &mut print_options) + return exec::exec_from_repl(&ctx, &mut print_options) .await .map_err(|e| DataFusionError::External(Box::new(e))); } if !files.is_empty() { - exec::exec_from_files(&mut ctx, files, &print_options).await?; + exec::exec_from_files(&ctx, files, &print_options).await?; } if !commands.is_empty() { - exec::exec_from_commands(&mut ctx, commands, &print_options).await?; + exec::exec_from_commands(&ctx, commands, &print_options).await?; } Ok(()) diff --git a/datafusion-examples/examples/catalog.rs b/datafusion-examples/examples/catalog.rs index f9ead592c7ea..f770056026ed 100644 --- a/datafusion-examples/examples/catalog.rs +++ b/datafusion-examples/examples/catalog.rs @@ -44,7 +44,7 @@ async fn main() -> Result<()> { let dir_a = prepare_example_data()?; let dir_b = prepare_example_data()?; - let mut ctx = SessionContext::new(); + let ctx = SessionContext::new(); let state = ctx.state(); let catlist = Arc::new(CustomCatalogProviderList::new()); diff --git a/datafusion/core/benches/filter_query_sql.rs b/datafusion/core/benches/filter_query_sql.rs index 01adc357b39a..0e09ae09d7c2 100644 --- a/datafusion/core/benches/filter_query_sql.rs +++ b/datafusion/core/benches/filter_query_sql.rs @@ -27,7 +27,7 @@ use futures::executor::block_on; use std::sync::Arc; use tokio::runtime::Runtime; -async fn query(ctx: &mut SessionContext, sql: &str) { +async fn query(ctx: &SessionContext, sql: &str) { let rt = Runtime::new().unwrap(); // execute the query @@ -70,25 +70,25 @@ fn criterion_benchmark(c: &mut Criterion) { let batch_size = 4096; // 2^12 c.bench_function("filter_array", |b| { - let mut ctx = create_context(array_len, batch_size).unwrap(); - b.iter(|| block_on(query(&mut ctx, "select f32, f64 from t where f32 >= f64"))) + let ctx = create_context(array_len, batch_size).unwrap(); + b.iter(|| block_on(query(&ctx, "select f32, f64 from t where f32 >= f64"))) }); c.bench_function("filter_scalar", |b| { - let mut ctx = create_context(array_len, batch_size).unwrap(); + let ctx = create_context(array_len, batch_size).unwrap(); b.iter(|| { block_on(query( - &mut ctx, + &ctx, "select f32, f64 from t where f32 >= 250 and f64 > 250", )) }) }); c.bench_function("filter_scalar in list", |b| { - let mut ctx = create_context(array_len, batch_size).unwrap(); + let ctx = create_context(array_len, batch_size).unwrap(); b.iter(|| { block_on(query( - &mut ctx, + &ctx, "select f32, f64 from t where f32 in (10, 20, 30, 40)", )) }) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index c4c5a4aa0834..cc1a63cc05f7 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1550,7 +1550,7 @@ impl DataFrame { /// # #[tokio::main] /// # async fn main() -> Result<()> { /// # use datafusion_common::ScalarValue; - /// let mut ctx = SessionContext::new(); + /// let ctx = SessionContext::new(); /// # ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?; /// let results = ctx /// .sql("SELECT a FROM example WHERE b = $1") @@ -2649,8 +2649,8 @@ mod tests { #[tokio::test] async fn registry() -> Result<()> { - let mut ctx = SessionContext::new(); - register_aggregate_csv(&mut ctx, "aggregate_test_100").await?; + let ctx = SessionContext::new(); + register_aggregate_csv(&ctx, "aggregate_test_100").await?; // declare the udf let my_fn: ScalarFunctionImplementation = @@ -2783,8 +2783,8 @@ mod tests { /// Create a logical plan from a SQL query async fn create_plan(sql: &str) -> Result { - let mut ctx = SessionContext::new(); - register_aggregate_csv(&mut ctx, "aggregate_test_100").await?; + let ctx = SessionContext::new(); + register_aggregate_csv(&ctx, "aggregate_test_100").await?; Ok(ctx.sql(sql).await?.into_unoptimized_plan()) } @@ -3147,9 +3147,9 @@ mod tests { "datafusion.sql_parser.enable_ident_normalization".to_owned(), "false".to_owned(), )]))?; - let mut ctx = SessionContext::new_with_config(config); + let ctx = SessionContext::new_with_config(config); let name = "aggregate_test_100"; - register_aggregate_csv(&mut ctx, name).await?; + register_aggregate_csv(&ctx, name).await?; let df = ctx.table(name); let df = df diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 1abb550f5c98..2a23f045f3b2 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -189,14 +189,14 @@ mod tests { async fn write_parquet_with_small_rg_size() -> Result<()> { // This test verifies writing a parquet file with small rg size // relative to datafusion.execution.batch_size does not panic - let mut ctx = SessionContext::new_with_config( - SessionConfig::from_string_hash_map(HashMap::from_iter( + let ctx = SessionContext::new_with_config(SessionConfig::from_string_hash_map( + HashMap::from_iter( [("datafusion.execution.batch_size", "10")] .iter() .map(|(s1, s2)| (s1.to_string(), s2.to_string())), - ))?, - ); - register_aggregate_csv(&mut ctx, "aggregate_test_100").await?; + ), + )?); + register_aggregate_csv(&ctx, "aggregate_test_100").await?; let test_df = ctx.table("aggregate_test_100").await?; let output_path = "file://local/test.parquet"; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c883b7a47fbb..c63ffddd81b3 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -179,7 +179,7 @@ where /// # use datafusion::{error::Result, assert_batches_eq}; /// # #[tokio::main] /// # async fn main() -> Result<()> { -/// let mut ctx = SessionContext::new(); +/// let ctx = SessionContext::new(); /// ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?; /// let results = ctx /// .sql("SELECT a, min(b) FROM example GROUP BY a LIMIT 100") @@ -369,7 +369,7 @@ impl SessionContext { /// # use datafusion_execution::object_store::ObjectStoreUrl; /// let object_store_url = ObjectStoreUrl::parse("file://").unwrap(); /// let object_store = object_store::local::LocalFileSystem::new(); - /// let mut ctx = SessionContext::new(); + /// let ctx = SessionContext::new(); /// // All files with the file:// url prefix will be read from the local file system /// ctx.register_object_store(object_store_url.as_ref(), Arc::new(object_store)); /// ``` @@ -452,7 +452,7 @@ impl SessionContext { /// # use datafusion::{error::Result, assert_batches_eq}; /// # #[tokio::main] /// # async fn main() -> Result<()> { - /// let mut ctx = SessionContext::new(); + /// let ctx = SessionContext::new(); /// ctx /// .sql("CREATE TABLE foo (x INTEGER)") /// .await? @@ -480,7 +480,7 @@ impl SessionContext { /// # use datafusion::physical_plan::collect; /// # #[tokio::main] /// # async fn main() -> Result<()> { - /// let mut ctx = SessionContext::new(); + /// let ctx = SessionContext::new(); /// let options = SQLOptions::new() /// .with_allow_ddl(false); /// let err = ctx.sql_with_options("CREATE TABLE foo (x INTEGER)", options) @@ -1357,7 +1357,7 @@ impl SessionContext { } /// Register [`CatalogProviderList`] in [`SessionState`] - pub fn register_catalog_list(&mut self, catalog_list: Arc) { + pub fn register_catalog_list(&self, catalog_list: Arc) { self.state.write().register_catalog_list(catalog_list) } @@ -1386,15 +1386,18 @@ impl FunctionRegistry for SessionContext { fn udwf(&self, name: &str) -> Result> { self.state.read().udwf(name) } + fn register_udf(&mut self, udf: Arc) -> Result>> { self.state.write().register_udf(udf) } + fn register_udaf( &mut self, udaf: Arc, ) -> Result>> { self.state.write().register_udaf(udaf) } + fn register_udwf(&mut self, udwf: Arc) -> Result>> { self.state.write().register_udwf(udwf) } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 9610a7f20364..937344ef5e4e 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -112,7 +112,7 @@ pub fn aggr_test_schema() -> SchemaRef { /// Register session context for the aggregate_test_100.csv file pub async fn register_aggregate_csv( - ctx: &mut SessionContext, + ctx: &SessionContext, table_name: &str, ) -> Result<()> { let schema = aggr_test_schema(); @@ -128,8 +128,8 @@ pub async fn register_aggregate_csv( /// Create a table from the aggregate_test_100.csv file with the specified name pub async fn test_table_with_name(name: &str) -> Result { - let mut ctx = SessionContext::new(); - register_aggregate_csv(&mut ctx, name).await?; + let ctx = SessionContext::new(); + register_aggregate_csv(&ctx, name).await?; ctx.table(name).await } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 47804b927e64..1aa33fc75e5d 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -103,7 +103,7 @@ use datafusion_optimizer::AnalyzerRule; /// Execute the specified sql and return the resulting record batches /// pretty printed as a String. -async fn exec_sql(ctx: &mut SessionContext, sql: &str) -> Result { +async fn exec_sql(ctx: &SessionContext, sql: &str) -> Result { let df = ctx.sql(sql).await?; let batches = df.collect().await?; pretty_format_batches(&batches) @@ -112,25 +112,25 @@ async fn exec_sql(ctx: &mut SessionContext, sql: &str) -> Result { } /// Create a test table. -async fn setup_table(mut ctx: SessionContext) -> Result { +async fn setup_table(ctx: SessionContext) -> Result { let sql = "CREATE EXTERNAL TABLE sales(customer_id VARCHAR, revenue BIGINT) STORED AS CSV location 'tests/data/customer.csv'"; let expected = vec!["++", "++"]; - let s = exec_sql(&mut ctx, sql).await?; + let s = exec_sql(&ctx, sql).await?; let actual = s.lines().collect::>(); assert_eq!(expected, actual, "Creating table"); Ok(ctx) } -async fn setup_table_without_schemas(mut ctx: SessionContext) -> Result { +async fn setup_table_without_schemas(ctx: SessionContext) -> Result { let sql = "CREATE EXTERNAL TABLE sales STORED AS CSV location 'tests/data/customer.csv'"; let expected = vec!["++", "++"]; - let s = exec_sql(&mut ctx, sql).await?; + let s = exec_sql(&ctx, sql).await?; let actual = s.lines().collect::>(); assert_eq!(expected, actual, "Creating table"); @@ -146,7 +146,7 @@ const QUERY2: &str = "SELECT 42, arrow_typeof(42)"; // Run the query using the specified execution context and compare it // to the known result -async fn run_and_compare_query(mut ctx: SessionContext, description: &str) -> Result<()> { +async fn run_and_compare_query(ctx: SessionContext, description: &str) -> Result<()> { let expected = vec![ "+-------------+---------+", "| customer_id | revenue |", @@ -157,7 +157,7 @@ async fn run_and_compare_query(mut ctx: SessionContext, description: &str) -> Re "+-------------+---------+", ]; - let s = exec_sql(&mut ctx, QUERY).await?; + let s = exec_sql(&ctx, QUERY).await?; let actual = s.lines().collect::>(); assert_eq!( @@ -174,7 +174,7 @@ async fn run_and_compare_query(mut ctx: SessionContext, description: &str) -> Re // Run the query using the specified execution context and compare it // to the known result async fn run_and_compare_query_with_analyzer_rule( - mut ctx: SessionContext, + ctx: SessionContext, description: &str, ) -> Result<()> { let expected = vec![ @@ -185,7 +185,7 @@ async fn run_and_compare_query_with_analyzer_rule( "+------------+--------------------------+", ]; - let s = exec_sql(&mut ctx, QUERY2).await?; + let s = exec_sql(&ctx, QUERY2).await?; let actual = s.lines().collect::>(); assert_eq!( @@ -202,7 +202,7 @@ async fn run_and_compare_query_with_analyzer_rule( // Run the query using the specified execution context and compare it // to the known result async fn run_and_compare_query_with_auto_schemas( - mut ctx: SessionContext, + ctx: SessionContext, description: &str, ) -> Result<()> { let expected = vec![ @@ -215,7 +215,7 @@ async fn run_and_compare_query_with_auto_schemas( "+----------+----------+", ]; - let s = exec_sql(&mut ctx, QUERY1).await?; + let s = exec_sql(&ctx, QUERY1).await?; let actual = s.lines().collect::>(); assert_eq!( @@ -262,13 +262,13 @@ async fn topk_query() -> Result<()> { #[tokio::test] // Run EXPLAIN PLAN and show the plan was in fact rewritten async fn topk_plan() -> Result<()> { - let mut ctx = setup_table(make_topk_context()).await?; + let ctx = setup_table(make_topk_context()).await?; let mut expected = ["| logical_plan after topk | TopK: k=3 |", "| | TableScan: sales projection=[customer_id,revenue] |"].join("\n"); let explain_query = format!("EXPLAIN VERBOSE {QUERY}"); - let actual_output = exec_sql(&mut ctx, &explain_query).await?; + let actual_output = exec_sql(&ctx, &explain_query).await?; // normalize newlines (output on windows uses \r\n) let mut actual_output = actual_output.replace("\r\n", "\n"); diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index b96398ef217f..e5c226418441 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -592,7 +592,9 @@ async fn roundtrip_logical_plan_copy_to_parquet() -> Result<()> { // Set specific Parquet format options let mut key_value_metadata = HashMap::new(); key_value_metadata.insert("test".to_string(), Some("test".to_string())); - parquet_format.key_value_metadata = key_value_metadata.clone(); + parquet_format + .key_value_metadata + .clone_from(&key_value_metadata); parquet_format.global.allow_single_file_parallelism = false; parquet_format.global.created_by = "test".to_string(); diff --git a/docs/source/library-user-guide/custom-table-providers.md b/docs/source/library-user-guide/custom-table-providers.md index a250e880913c..f86cea0bda95 100644 --- a/docs/source/library-user-guide/custom-table-providers.md +++ b/docs/source/library-user-guide/custom-table-providers.md @@ -146,7 +146,7 @@ For filters that can be pushed down, they'll be passed to the `scan` method as t In order to use the custom table provider, we need to register it with DataFusion. This is done by creating a `TableProvider` and registering it with the `SessionContext`. ```rust -let mut ctx = SessionContext::new(); +let ctx = SessionContext::new(); let custom_table_provider = CustomDataSource::new(); ctx.register_table("custom_table", Arc::new(custom_table_provider)); From 053795cd4ec5cbac022fbea767b79f8da1424060 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Thu, 8 Aug 2024 09:48:09 +0800 Subject: [PATCH 31/34] Improve nested loop join code (#11863) * Improve nested loop join code * fmt --- .../src/joins/nested_loop_join.rs | 90 +++++++++---------- 1 file changed, 41 insertions(+), 49 deletions(-) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 9f1465c2d7c1..d69d818331be 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -47,7 +47,7 @@ use arrow::compute::concat_batches; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; -use datafusion_common::{exec_err, JoinSide, Result, Statistics}; +use datafusion_common::{exec_datafusion_err, JoinSide, Result, Statistics}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::JoinType; @@ -562,62 +562,54 @@ fn join_left_and_right_batch( schema: &Schema, visited_left_side: &SharedBitmapBuilder, ) -> Result { - let indices_result = (0..left_batch.num_rows()) + let indices = (0..left_batch.num_rows()) .map(|left_row_index| { build_join_indices(left_row_index, right_batch, left_batch, filter) }) - .collect::>>(); + .collect::>>() + .map_err(|e| { + exec_datafusion_err!( + "Fail to build join indices in NestedLoopJoinExec, error:{e}" + ) + })?; let mut left_indices_builder = UInt64Builder::new(); let mut right_indices_builder = UInt32Builder::new(); - let left_right_indices = match indices_result { - Err(err) => { - exec_err!("Fail to build join indices in NestedLoopJoinExec, error:{err}") - } - Ok(indices) => { - for (left_side, right_side) in indices { - left_indices_builder - .append_values(left_side.values(), &vec![true; left_side.len()]); - right_indices_builder - .append_values(right_side.values(), &vec![true; right_side.len()]); - } - Ok(( - left_indices_builder.finish(), - right_indices_builder.finish(), - )) - } - }; - match left_right_indices { - Ok((left_side, right_side)) => { - // set the left bitmap - // and only full join need the left bitmap - if need_produce_result_in_final(join_type) { - let mut bitmap = visited_left_side.lock(); - left_side.iter().flatten().for_each(|x| { - bitmap.set_bit(x as usize, true); - }); - } - // adjust the two side indices base on the join type - let (left_side, right_side) = adjust_indices_by_join_type( - left_side, - right_side, - 0..right_batch.num_rows(), - join_type, - false, - ); + for (left_side, right_side) in indices { + left_indices_builder + .append_values(left_side.values(), &vec![true; left_side.len()]); + right_indices_builder + .append_values(right_side.values(), &vec![true; right_side.len()]); + } - build_batch_from_indices( - schema, - left_batch, - right_batch, - &left_side, - &right_side, - column_indices, - JoinSide::Left, - ) - } - Err(e) => Err(e), + let left_side = left_indices_builder.finish(); + let right_side = right_indices_builder.finish(); + // set the left bitmap + // and only full join need the left bitmap + if need_produce_result_in_final(join_type) { + let mut bitmap = visited_left_side.lock(); + left_side.iter().flatten().for_each(|x| { + bitmap.set_bit(x as usize, true); + }); } + // adjust the two side indices base on the join type + let (left_side, right_side) = adjust_indices_by_join_type( + left_side, + right_side, + 0..right_batch.num_rows(), + join_type, + false, + ); + + build_batch_from_indices( + schema, + left_batch, + right_batch, + &left_side, + &right_side, + column_indices, + JoinSide::Left, + ) } fn get_final_indices_from_shared_bitmap( From 0bbce5dce29df1123b0ab87a8907482c72d284c1 Mon Sep 17 00:00:00 2001 From: Douglas Anderson Date: Wed, 7 Aug 2024 20:32:57 -0600 Subject: [PATCH 32/34] [Minor]: Refactor to use Result.transpose() (#11882) `Result.transpose()` converts `Result>` to `Option>`. > Ok(None) will be mapped to None. Ok(Some(_)) and Err(_) will be mapped to Some(Ok(_)) and Some(Err(_)). - https://doc.rust-lang.org/std/result/enum.Result.html#method.transpose --- .../core/src/datasource/physical_plan/arrow_file.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index e720b4efff6f..a1ee6fbe1341 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -331,11 +331,9 @@ impl FileOpener for ArrowOpener { .into_iter() .zip(recordbatch_results) .filter_map(move |(block, data)| { - match decoder.read_record_batch(&block, &data.into()) { - Ok(Some(record_batch)) => Some(Ok(record_batch)), - Ok(None) => None, - Err(err) => Some(Err(err)), - } + decoder + .read_record_batch(&block, &data.into()) + .transpose() }), ) .boxed()) From 2521043ddcb3895a2010b8e328f3fa10f77fc094 Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Thu, 8 Aug 2024 12:27:25 +0100 Subject: [PATCH 33/34] support `ANY()` op (#11849) * support ANY() op * use ExprPlanner * revert test changes * add planner tests * minimise diff * fix tests :fingers_crossed: * move error test to slt --- datafusion/expr/src/planner.rs | 7 +++ datafusion/functions-nested/src/array_has.rs | 22 +++++----- datafusion/functions-nested/src/planner.rs | 20 +++++++-- datafusion/sql/src/expr/mod.rs | 46 ++++++++++++++++---- datafusion/sqllogictest/test_files/array.slt | 19 ++++++++ 5 files changed, 92 insertions(+), 22 deletions(-) diff --git a/datafusion/expr/src/planner.rs b/datafusion/expr/src/planner.rs index c775427df138..24f589c41582 100644 --- a/datafusion/expr/src/planner.rs +++ b/datafusion/expr/src/planner.rs @@ -197,6 +197,13 @@ pub trait ExprPlanner: Send + Sync { "Default planner compound identifier hasn't been implemented for ExprPlanner" ) } + + /// Plans `ANY` expression, e.g., `expr = ANY(array_expr)` + /// + /// Returns origin binary expression if not possible + fn plan_any(&self, expr: RawBinaryExpr) -> Result> { + Ok(PlannerResult::Original(expr)) + } } /// An operator with two arguments to plan diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index bdda5a565947..fe1df2579932 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -34,19 +34,19 @@ use std::sync::Arc; // Create static instances of ScalarUDFs for each function make_udf_expr_and_func!(ArrayHas, array_has, - first_array second_array, // arg name + haystack_array element, // arg names "returns true, if the element appears in the first array, otherwise false.", // doc array_has_udf // internal function name ); make_udf_expr_and_func!(ArrayHasAll, array_has_all, - first_array second_array, // arg name + haystack_array needle_array, // arg names "returns true if each element of the second array appears in the first array; otherwise, it returns false.", // doc array_has_all_udf // internal function name ); make_udf_expr_and_func!(ArrayHasAny, array_has_any, - first_array second_array, // arg name + haystack_array needle_array, // arg names "returns true if at least one element of the second array appears in the first array; otherwise, it returns false.", // doc array_has_any_udf // internal function name ); @@ -262,26 +262,26 @@ enum ComparisonType { } fn general_array_has_dispatch( - array: &ArrayRef, - sub_array: &ArrayRef, + haystack: &ArrayRef, + needle: &ArrayRef, comparison_type: ComparisonType, ) -> Result { let array = if comparison_type == ComparisonType::Single { - let arr = as_generic_list_array::(array)?; - check_datatypes("array_has", &[arr.values(), sub_array])?; + let arr = as_generic_list_array::(haystack)?; + check_datatypes("array_has", &[arr.values(), needle])?; arr } else { - check_datatypes("array_has", &[array, sub_array])?; - as_generic_list_array::(array)? + check_datatypes("array_has", &[haystack, needle])?; + as_generic_list_array::(haystack)? }; let mut boolean_builder = BooleanArray::builder(array.len()); let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; - let element = Arc::clone(sub_array); + let element = Arc::clone(needle); let sub_array = if comparison_type != ComparisonType::Single { - as_generic_list_array::(sub_array)? + as_generic_list_array::(needle)? } else { array }; diff --git a/datafusion/functions-nested/src/planner.rs b/datafusion/functions-nested/src/planner.rs index f980362105a1..4cd8faa3ca98 100644 --- a/datafusion/functions-nested/src/planner.rs +++ b/datafusion/functions-nested/src/planner.rs @@ -17,7 +17,7 @@ //! SQL planning extensions like [`NestedFunctionPlanner`] and [`FieldAccessPlanner`] -use datafusion_common::{exec_err, utils::list_ndims, DFSchema, Result}; +use datafusion_common::{plan_err, utils::list_ndims, DFSchema, Result}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ planner::{ExprPlanner, PlannerResult, RawBinaryExpr, RawFieldAccessExpr}, @@ -28,7 +28,7 @@ use datafusion_functions_aggregate::nth_value::nth_value_udaf; use crate::map::map_udf; use crate::{ - array_has::array_has_all, + array_has::{array_has_all, array_has_udf}, expr_fn::{array_append, array_concat, array_prepend}, extract::{array_element, array_slice}, make_array::make_array, @@ -102,7 +102,7 @@ impl ExprPlanner for NestedFunctionPlanner { fn plan_make_map(&self, args: Vec) -> Result>> { if args.len() % 2 != 0 { - return exec_err!("make_map requires an even number of arguments"); + return plan_err!("make_map requires an even number of arguments"); } let (keys, values): (Vec<_>, Vec<_>) = @@ -114,6 +114,20 @@ impl ExprPlanner for NestedFunctionPlanner { ScalarFunction::new_udf(map_udf(), vec![keys, values]), ))) } + + fn plan_any(&self, expr: RawBinaryExpr) -> Result> { + if expr.op == sqlparser::ast::BinaryOperator::Eq { + Ok(PlannerResult::Planned(Expr::ScalarFunction( + ScalarFunction::new_udf( + array_has_udf(), + // left and right are reversed here so `needle=any(haystack)` -> `array_has(haystack, needle)` + vec![expr.right, expr.left], + ), + ))) + } else { + plan_err!("Unsupported AnyOp: '{}', only '=' is supported", expr.op) + } + } } pub struct FieldAccessPlanner; diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index b80ffb6aed3f..edb0002842a8 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -17,12 +17,12 @@ use arrow_schema::DataType; use arrow_schema::TimeUnit; -use datafusion_expr::planner::PlannerResult; -use datafusion_expr::planner::RawDictionaryExpr; -use datafusion_expr::planner::RawFieldAccessExpr; +use datafusion_expr::planner::{ + PlannerResult, RawBinaryExpr, RawDictionaryExpr, RawFieldAccessExpr, +}; use sqlparser::ast::{ - CastKind, DictionaryField, Expr as SQLExpr, MapEntry, StructField, Subscript, - TrimWhereField, Value, + BinaryOperator, CastKind, DictionaryField, Expr as SQLExpr, MapEntry, StructField, + Subscript, TrimWhereField, Value, }; use datafusion_common::{ @@ -104,13 +104,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { fn build_logical_expr( &self, - op: sqlparser::ast::BinaryOperator, + op: BinaryOperator, left: Expr, right: Expr, schema: &DFSchema, ) -> Result { // try extension planers - let mut binary_expr = datafusion_expr::planner::RawBinaryExpr { op, left, right }; + let mut binary_expr = RawBinaryExpr { op, left, right }; for planner in self.context_provider.get_expr_planners() { match planner.plan_binary_op(binary_expr, schema)? { PlannerResult::Planned(expr) => { @@ -122,7 +122,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - let datafusion_expr::planner::RawBinaryExpr { op, left, right } = binary_expr; + let RawBinaryExpr { op, left, right } = binary_expr; Ok(Expr::BinaryExpr(BinaryExpr::new( Box::new(left), self.parse_sql_binary_op(op)?, @@ -631,6 +631,36 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Map(map) => { self.try_plan_map_literal(map.entries, schema, planner_context) } + SQLExpr::AnyOp { + left, + compare_op, + right, + } => { + let mut binary_expr = RawBinaryExpr { + op: compare_op, + left: self.sql_expr_to_logical_expr( + *left, + schema, + planner_context, + )?, + right: self.sql_expr_to_logical_expr( + *right, + schema, + planner_context, + )?, + }; + for planner in self.context_provider.get_expr_planners() { + match planner.plan_any(binary_expr)? { + PlannerResult::Planned(expr) => { + return Ok(expr); + } + PlannerResult::Original(expr) => { + binary_expr = expr; + } + } + } + not_impl_err!("AnyOp not supported by ExprPlanner: {binary_expr:?}") + } _ => not_impl_err!("Unsupported ast node in sqltorel: {sql:?}"), } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index f2972e4c14c2..b71bc765ba37 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -5351,6 +5351,25 @@ true false true false false false true true false false true false true #---- #true false true false false false true true false false true false true +# any operator +query ? +select column3 from arrays where 'L'=any(column3); +---- +[L, o, r, e, m] + +query I +select count(*) from arrays where 'L'=any(column3); +---- +1 + +query I +select count(*) from arrays where 'X'=any(column3); +---- +0 + +query error DataFusion error: Error during planning: Unsupported AnyOp: '>', only '=' is supported +select count(*) from arrays where 'X'>any(column3); + ## array_distinct #TODO: https://github.com/apache/datafusion/issues/7142 From 1f35b03426b00e176518b6791cd73a181473dc1a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 8 Aug 2024 17:23:48 +0200 Subject: [PATCH 34/34] Sync rust docs params for CI and dev (#11890) Since a4ac0829ecf63b3640315835b1374211dfadd953 commit there was a discrepancy between rust.yml GitHub workflow and the `dev/rust_lint.sh` script behavior. Sync the behaviors. Reuse common script to prevent future discrepancies. --- .github/workflows/rust.yml | 6 +----- ci/scripts/rust_docs.sh | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 00d31765e77d..809f3acd8374 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -233,11 +233,7 @@ jobs: with: rust-version: stable - name: Run cargo doc - run: | - export RUSTDOCFLAGS="-D warnings" - cargo doc --document-private-items --no-deps --workspace - cd datafusion-cli - cargo doc --document-private-items --no-deps + run: ci/scripts/rust_docs.sh linux-wasm-pack: name: build with wasm-pack diff --git a/ci/scripts/rust_docs.sh b/ci/scripts/rust_docs.sh index cf83b80b5132..5c93711b6fb6 100755 --- a/ci/scripts/rust_docs.sh +++ b/ci/scripts/rust_docs.sh @@ -18,7 +18,7 @@ # under the License. set -ex -export RUSTDOCFLAGS="-D warnings -A rustdoc::private-intra-doc-links" +export RUSTDOCFLAGS="-D warnings" cargo doc --document-private-items --no-deps --workspace cd datafusion-cli cargo doc --document-private-items --no-deps