From 378b9eecd4a77386a59953209f75fc5c192d7af4 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Mon, 17 Jun 2024 17:43:20 +0800 Subject: [PATCH 01/54] chore: Improve performance of Parquet statistics conversion (#10932) --- .../physical_plan/parquet/statistics.rs | 32 +++---------------- 1 file changed, 4 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index a2e0d8fa66be..327a516f1af1 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -303,24 +303,12 @@ macro_rules! get_statistics { ))), DataType::Int8 => Ok(Arc::new(Int8Array::from_iter( [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - if let Ok(v) = i8::try_from(*x) { - Some(v) - } else { - None - } - }) + x.and_then(|x| i8::try_from(*x).ok()) }), ))), DataType::Int16 => Ok(Arc::new(Int16Array::from_iter( [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - if let Ok(v) = i16::try_from(*x) { - Some(v) - } else { - None - } - }) + x.and_then(|x| i16::try_from(*x).ok()) }), ))), DataType::Int32 => Ok(Arc::new(Int32Array::from_iter( @@ -331,24 +319,12 @@ macro_rules! get_statistics { ))), DataType::UInt8 => Ok(Arc::new(UInt8Array::from_iter( [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - if let Ok(v) = u8::try_from(*x) { - Some(v) - } else { - None - } - }) + x.and_then(|x| u8::try_from(*x).ok()) }), ))), DataType::UInt16 => Ok(Arc::new(UInt16Array::from_iter( [<$stat_type_prefix Int32StatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - if let Ok(v) = u16::try_from(*x) { - Some(v) - } else { - None - } - }) + x.and_then(|x| u16::try_from(*x).ok()) }), ))), DataType::UInt32 => Ok(Arc::new(UInt32Array::from_iter( From c4fd7545ba7719d6d12473694fcdf6f34d25b8cb Mon Sep 17 00:00:00 2001 From: Leonardo Yvens Date: Mon, 17 Jun 2024 12:17:58 +0100 Subject: [PATCH 02/54] Add catalog::resolve_table_references (#10876) * resolve information_schema references only when necessary * add `catalog::resolve_table_references` as a public utility * collect CTEs separately in resolve_table_references * test CTE name shadowing * handle CTE name shadowing in resolve_table_references * handle unions, recursive and nested CTEs in resolve_table_references --- datafusion/core/src/catalog/mod.rs | 239 +++++++++++++++++- .../core/src/execution/session_state.rs | 96 +------ datafusion/sqllogictest/test_files/cte.slt | 7 + 3 files changed, 256 insertions(+), 86 deletions(-) diff --git a/datafusion/core/src/catalog/mod.rs b/datafusion/core/src/catalog/mod.rs index 209d9b2af297..53b133339924 100644 --- a/datafusion/core/src/catalog/mod.rs +++ b/datafusion/core/src/catalog/mod.rs @@ -27,6 +27,8 @@ use crate::catalog::schema::SchemaProvider; use dashmap::DashMap; use datafusion_common::{exec_err, not_impl_err, Result}; use std::any::Any; +use std::collections::BTreeSet; +use std::ops::ControlFlow; use std::sync::Arc; /// Represent a list of named [`CatalogProvider`]s. @@ -157,11 +159,11 @@ impl CatalogProviderList for MemoryCatalogProviderList { /// access required to read table details (e.g. statistics). /// /// The pattern that DataFusion itself uses to plan SQL queries is to walk over -/// the query to [find all schema / table references in an `async` function], +/// the query to [find all table references], /// performing required remote catalog in parallel, and then plans the query /// using that snapshot. /// -/// [find all schema / table references in an `async` function]: crate::execution::context::SessionState::resolve_table_references +/// [find all table references]: resolve_table_references /// /// # Example Catalog Implementations /// @@ -295,6 +297,182 @@ impl CatalogProvider for MemoryCatalogProvider { } } +/// Collects all tables and views referenced in the SQL statement. CTEs are collected separately. +/// This can be used to determine which tables need to be in the catalog for a query to be planned. +/// +/// # Returns +/// +/// A `(table_refs, ctes)` tuple, the first element contains table and view references and the second +/// element contains any CTE aliases that were defined and possibly referenced. +/// +/// ## Example +/// +/// ``` +/// # use datafusion_sql::parser::DFParser; +/// # use datafusion::catalog::resolve_table_references; +/// let query = "SELECT a FROM foo where x IN (SELECT y FROM bar)"; +/// let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); +/// let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); +/// assert_eq!(table_refs.len(), 2); +/// assert_eq!(table_refs[0].to_string(), "bar"); +/// assert_eq!(table_refs[1].to_string(), "foo"); +/// assert_eq!(ctes.len(), 0); +/// ``` +/// +/// ## Example with CTEs +/// +/// ``` +/// # use datafusion_sql::parser::DFParser; +/// # use datafusion::catalog::resolve_table_references; +/// let query = "with my_cte as (values (1), (2)) SELECT * from my_cte;"; +/// let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); +/// let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); +/// assert_eq!(table_refs.len(), 0); +/// assert_eq!(ctes.len(), 1); +/// assert_eq!(ctes[0].to_string(), "my_cte"); +/// ``` +pub fn resolve_table_references( + statement: &datafusion_sql::parser::Statement, + enable_ident_normalization: bool, +) -> datafusion_common::Result<(Vec, Vec)> { + use crate::sql::planner::object_name_to_table_reference; + use datafusion_sql::parser::{ + CopyToSource, CopyToStatement, Statement as DFStatement, + }; + use information_schema::INFORMATION_SCHEMA; + use information_schema::INFORMATION_SCHEMA_TABLES; + use sqlparser::ast::*; + + struct RelationVisitor { + relations: BTreeSet, + all_ctes: BTreeSet, + ctes_in_scope: Vec, + } + + impl RelationVisitor { + /// Record the reference to `relation`, if it's not a CTE reference. + fn insert_relation(&mut self, relation: &ObjectName) { + if !self.relations.contains(relation) + && !self.ctes_in_scope.contains(relation) + { + self.relations.insert(relation.clone()); + } + } + } + + impl Visitor for RelationVisitor { + type Break = (); + + fn pre_visit_relation(&mut self, relation: &ObjectName) -> ControlFlow<()> { + self.insert_relation(relation); + ControlFlow::Continue(()) + } + + fn pre_visit_query(&mut self, q: &Query) -> ControlFlow { + if let Some(with) = &q.with { + for cte in &with.cte_tables { + // The non-recursive CTE name is not in scope when evaluating the CTE itself, so this is valid: + // `WITH t AS (SELECT * FROM t) SELECT * FROM t` + // Where the first `t` refers to a predefined table. So we are careful here + // to visit the CTE first, before putting it in scope. + if !with.recursive { + // This is a bit hackish as the CTE will be visited again as part of visiting `q`, + // but thankfully `insert_relation` is idempotent. + cte.visit(self); + } + self.ctes_in_scope + .push(ObjectName(vec![cte.alias.name.clone()])); + } + } + ControlFlow::Continue(()) + } + + fn post_visit_query(&mut self, q: &Query) -> ControlFlow { + if let Some(with) = &q.with { + for _ in &with.cte_tables { + // Unwrap: We just pushed these in `pre_visit_query` + self.all_ctes.insert(self.ctes_in_scope.pop().unwrap()); + } + } + ControlFlow::Continue(()) + } + + fn pre_visit_statement(&mut self, statement: &Statement) -> ControlFlow<()> { + if let Statement::ShowCreate { + obj_type: ShowCreateObject::Table | ShowCreateObject::View, + obj_name, + } = statement + { + self.insert_relation(obj_name) + } + + // SHOW statements will later be rewritten into a SELECT from the information_schema + let requires_information_schema = matches!( + statement, + Statement::ShowFunctions { .. } + | Statement::ShowVariable { .. } + | Statement::ShowStatus { .. } + | Statement::ShowVariables { .. } + | Statement::ShowCreate { .. } + | Statement::ShowColumns { .. } + | Statement::ShowTables { .. } + | Statement::ShowCollation { .. } + ); + if requires_information_schema { + for s in INFORMATION_SCHEMA_TABLES { + self.relations.insert(ObjectName(vec![ + Ident::new(INFORMATION_SCHEMA), + Ident::new(*s), + ])); + } + } + ControlFlow::Continue(()) + } + } + + let mut visitor = RelationVisitor { + relations: BTreeSet::new(), + all_ctes: BTreeSet::new(), + ctes_in_scope: vec![], + }; + + fn visit_statement(statement: &DFStatement, visitor: &mut RelationVisitor) { + match statement { + DFStatement::Statement(s) => { + let _ = s.as_ref().visit(visitor); + } + DFStatement::CreateExternalTable(table) => { + visitor + .relations + .insert(ObjectName(vec![Ident::from(table.name.as_str())])); + } + DFStatement::CopyTo(CopyToStatement { source, .. }) => match source { + CopyToSource::Relation(table_name) => { + visitor.insert_relation(table_name); + } + CopyToSource::Query(query) => { + query.visit(visitor); + } + }, + DFStatement::Explain(explain) => visit_statement(&explain.statement, visitor), + } + } + + visit_statement(statement, &mut visitor); + + let table_refs = visitor + .relations + .into_iter() + .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) + .collect::>()?; + let ctes = visitor + .all_ctes + .into_iter() + .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) + .collect::>()?; + Ok((table_refs, ctes)) +} + #[cfg(test)] mod tests { use super::*; @@ -363,4 +541,61 @@ mod tests { let cat = Arc::new(MemoryCatalogProvider::new()) as Arc; assert!(cat.deregister_schema("foo", false).unwrap().is_none()); } + + #[test] + fn resolve_table_references_shadowed_cte() { + use datafusion_sql::parser::DFParser; + + // An interesting edge case where the `t` name is used both as an ordinary table reference + // and as a CTE reference. + let query = "WITH t AS (SELECT * FROM t) SELECT * FROM t"; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 1); + assert_eq!(ctes.len(), 1); + assert_eq!(ctes[0].to_string(), "t"); + assert_eq!(table_refs[0].to_string(), "t"); + + // UNION is a special case where the CTE is not in scope for the second branch. + let query = "(with t as (select 1) select * from t) union (select * from t)"; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 1); + assert_eq!(ctes.len(), 1); + assert_eq!(ctes[0].to_string(), "t"); + assert_eq!(table_refs[0].to_string(), "t"); + + // Nested CTEs are also handled. + // Here the first `u` is a CTE, but the second `u` is a table reference. + // While `t` is always a CTE. + let query = "(with t as (with u as (select 1) select * from u) select * from u cross join t)"; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 1); + assert_eq!(ctes.len(), 2); + assert_eq!(ctes[0].to_string(), "t"); + assert_eq!(ctes[1].to_string(), "u"); + assert_eq!(table_refs[0].to_string(), "u"); + } + + #[test] + fn resolve_table_references_recursive_cte() { + use datafusion_sql::parser::DFParser; + + let query = " + WITH RECURSIVE nodes AS ( + SELECT 1 as id + UNION ALL + SELECT id + 1 as id + FROM nodes + WHERE id < 10 + ) + SELECT * FROM nodes + "; + let statement = DFParser::parse_sql(query).unwrap().pop_back().unwrap(); + let (table_refs, ctes) = resolve_table_references(&statement, true).unwrap(); + assert_eq!(table_refs.len(), 0); + assert_eq!(ctes.len(), 1); + assert_eq!(ctes[0].to_string(), "nodes"); + } } diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index fed101bd239b..1df77a1f9e0b 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -66,15 +66,12 @@ use datafusion_optimizer::{ use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; -use datafusion_sql::parser::{CopyToSource, CopyToStatement, DFParser, Statement}; -use datafusion_sql::planner::{ - object_name_to_table_reference, ContextProvider, ParserOptions, SqlToRel, -}; +use datafusion_sql::parser::{DFParser, Statement}; +use datafusion_sql::planner::{ContextProvider, ParserOptions, SqlToRel}; use sqlparser::dialect::dialect_from_str; use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet}; use std::fmt::Debug; -use std::ops::ControlFlow; use std::sync::Arc; use url::Url; use uuid::Uuid; @@ -493,91 +490,22 @@ impl SessionState { Ok(statement) } - /// Resolve all table references in the SQL statement. + /// Resolve all table references in the SQL statement. Does not include CTE references. + /// + /// See [`catalog::resolve_table_references`] for more information. + /// + /// [`catalog::resolve_table_references`]: crate::catalog::resolve_table_references pub fn resolve_table_references( &self, statement: &datafusion_sql::parser::Statement, ) -> datafusion_common::Result> { - use crate::catalog::information_schema::INFORMATION_SCHEMA_TABLES; - use datafusion_sql::parser::Statement as DFStatement; - use sqlparser::ast::*; - - // Getting `TableProviders` is async but planing is not -- thus pre-fetch - // table providers for all relations referenced in this query - let mut relations = hashbrown::HashSet::with_capacity(10); - - struct RelationVisitor<'a>(&'a mut hashbrown::HashSet); - - impl<'a> RelationVisitor<'a> { - /// Record that `relation` was used in this statement - fn insert(&mut self, relation: &ObjectName) { - self.0.get_or_insert_with(relation, |_| relation.clone()); - } - } - - impl<'a> Visitor for RelationVisitor<'a> { - type Break = (); - - fn pre_visit_relation(&mut self, relation: &ObjectName) -> ControlFlow<()> { - self.insert(relation); - ControlFlow::Continue(()) - } - - fn pre_visit_statement(&mut self, statement: &Statement) -> ControlFlow<()> { - if let Statement::ShowCreate { - obj_type: ShowCreateObject::Table | ShowCreateObject::View, - obj_name, - } = statement - { - self.insert(obj_name) - } - ControlFlow::Continue(()) - } - } - - let mut visitor = RelationVisitor(&mut relations); - fn visit_statement(statement: &DFStatement, visitor: &mut RelationVisitor<'_>) { - match statement { - DFStatement::Statement(s) => { - let _ = s.as_ref().visit(visitor); - } - DFStatement::CreateExternalTable(table) => { - visitor - .0 - .insert(ObjectName(vec![Ident::from(table.name.as_str())])); - } - DFStatement::CopyTo(CopyToStatement { source, .. }) => match source { - CopyToSource::Relation(table_name) => { - visitor.insert(table_name); - } - CopyToSource::Query(query) => { - query.visit(visitor); - } - }, - DFStatement::Explain(explain) => { - visit_statement(&explain.statement, visitor) - } - } - } - - visit_statement(statement, &mut visitor); - - // Always include information_schema if available - if self.config.information_schema() { - for s in INFORMATION_SCHEMA_TABLES { - relations.insert(ObjectName(vec![ - Ident::new(INFORMATION_SCHEMA), - Ident::new(*s), - ])); - } - } - let enable_ident_normalization = self.config.options().sql_parser.enable_ident_normalization; - relations - .into_iter() - .map(|x| object_name_to_table_reference(x, enable_ident_normalization)) - .collect::>() + let (table_refs, _) = crate::catalog::resolve_table_references( + statement, + enable_ident_normalization, + )?; + Ok(table_refs) } /// Convert an AST Statement into a LogicalPlan diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 1ff108cf6c5f..d8eaa51fc88a 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -828,3 +828,10 @@ SELECT * FROM non_recursive_cte, recursive_cte; ---- 1 1 1 3 + +# Name shadowing: +# The first `t` refers to the table, the second to the CTE. +query I +WITH t AS (SELECT * FROM t where t.a < 2) SELECT * FROM t +---- +1 \ No newline at end of file From a923c659cf932f6369f2d5257e5b99128b67091a Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Mon, 17 Jun 2024 19:22:55 +0800 Subject: [PATCH 03/54] feat: Add support for Int8 and Int16 data types in data page statistics (#10931) --- .../physical_plan/parquet/statistics.rs | 30 +++++++++++++++++++ .../core/tests/parquet/arrow_statistics.rs | 24 ++------------- 2 files changed, 33 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index 327a516f1af1..a2f17ca9b7a7 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -548,6 +548,8 @@ macro_rules! make_data_page_stats_iterator { }; } +make_data_page_stats_iterator!(MinInt32DataPageStatsIterator, min, Index::INT32, i32); +make_data_page_stats_iterator!(MaxInt32DataPageStatsIterator, max, Index::INT32, i32); make_data_page_stats_iterator!(MinInt64DataPageStatsIterator, min, Index::INT64, i64); make_data_page_stats_iterator!(MaxInt64DataPageStatsIterator, max, Index::INT64, i64); @@ -555,6 +557,29 @@ macro_rules! get_data_page_statistics { ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { paste! { match $data_type { + Some(DataType::Int8) => Ok(Arc::new( + Int8Array::from_iter( + [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| i8::try_from(x).ok()) + }) + }) + .flatten() + ) + )), + Some(DataType::Int16) => Ok(Arc::new( + Int16Array::from_iter( + [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| i16::try_from(x).ok()) + }) + }) + .flatten() + ) + )), + Some(DataType::Int32) => Ok(Arc::new(Int32Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten()))), Some(DataType::Int64) => Ok(Arc::new(Int64Array::from_iter([<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator).flatten()))), _ => unimplemented!() } @@ -642,6 +667,11 @@ where { let iter = iterator.flat_map(|(len, index)| match index { Index::NONE => vec![None; len], + Index::INT32(native_index) => native_index + .indexes + .iter() + .map(|x| x.null_count.map(|x| x as u64)) + .collect::>(), Index::INT64(native_index) => native_index .indexes .iter() diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index 6b8705441d12..87bd1372225f 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -550,16 +550,11 @@ async fn test_int_32() { // row counts are [5, 5, 5, 5] expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), column_name: "i32", - check: Check::RowGroup, + check: Check::Both, } .run(); } -// BUG: ignore this test for now -// https://github.com/apache/datafusion/issues/10585 -// Note that the file has 4 columns named "i8", "i16", "i32", "i64". -// - The tests on column i32 and i64 passed. -// - The tests on column i8 and i16 failed. #[tokio::test] async fn test_int_16() { // This creates a parquet files of 4 columns named "i8", "i16", "i32", "i64" @@ -573,16 +568,6 @@ async fn test_int_16() { Test { reader: &reader, // mins are [-5, -4, 0, 5] - // BUG: not sure why this returns same data but in Int32Array type even though I debugged and the columns name is "i16" an its data is Int16 - // My debugging tells me the bug is either at: - // 1. The new code to get "iter". See the code in this PR with - // // Get an iterator over the column statistics - // let iter = row_groups - // .iter() - // .map(|x| x.column(parquet_idx).statistics()); - // OR - // 2. in the function (and/or its marco) `pub(crate) fn min_statistics<'a, I: Iterator>>` here - // https://github.com/apache/datafusion/blob/ea023e2d4878240eece870cf4b346c7a0667aeed/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs#L179 expected_min: Arc::new(Int16Array::from(vec![-5, -4, 0, 5])), // panic here because the actual data is Int32Array // maxes are [-1, 0, 4, 9] expected_max: Arc::new(Int16Array::from(vec![-1, 0, 4, 9])), @@ -591,13 +576,11 @@ async fn test_int_16() { // row counts are [5, 5, 5, 5] expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), column_name: "i16", - check: Check::RowGroup, + check: Check::Both, } .run(); } -// BUG (same as above): ignore this test for now -// https://github.com/apache/datafusion/issues/10585 #[tokio::test] async fn test_int_8() { // This creates a parquet files of 4 columns named "i8", "i16", "i32", "i64" @@ -611,7 +594,6 @@ async fn test_int_8() { Test { reader: &reader, // mins are [-5, -4, 0, 5] - // BUG: not sure why this returns same data but in Int32Array even though I debugged and the columns name is "i8" an its data is Int8 expected_min: Arc::new(Int8Array::from(vec![-5, -4, 0, 5])), // panic here because the actual data is Int32Array // maxes are [-1, 0, 4, 9] expected_max: Arc::new(Int8Array::from(vec![-1, 0, 4, 9])), @@ -620,7 +602,7 @@ async fn test_int_8() { // row counts are [5, 5, 5, 5] expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), column_name: "i8", - check: Check::RowGroup, + check: Check::Both, } .run(); } From 2daadb75230e2c197d2915257a9637913fa2c2e6 Mon Sep 17 00:00:00 2001 From: Dharan Aditya Date: Mon, 17 Jun 2024 18:36:16 +0530 Subject: [PATCH 04/54] Convert BitAnd, BitOr, BitXor to UDAF (#10930) * remove bit and or xor from expr * remove bit and or xor from physical expr and proto * add proto regen changes * impl BitAnd, BitOr, BitXor UADF * add support for float * removing support for float * refactor helper macros * clippy'fy * simplify Bitwise operation * add documentation * formatting * fix lint issue * remove XorDistinct * update roundtrip_expr_api test * linting * support groups accumulator --- datafusion/expr/src/aggregate_function.rs | 20 - .../expr/src/type_coercion/aggregates.rs | 18 - .../functions-aggregate/src/bit_and_or_xor.rs | 458 ++++++++++++ datafusion/functions-aggregate/src/lib.rs | 7 + .../src/aggregate/bit_and_or_xor.rs | 695 ------------------ .../physical-expr/src/aggregate/build_in.rs | 78 +- datafusion/physical-expr/src/aggregate/mod.rs | 1 - .../physical-expr/src/expressions/mod.rs | 1 - datafusion/proto/proto/datafusion.proto | 6 +- datafusion/proto/src/generated/pbjson.rs | 9 - datafusion/proto/src/generated/prost.rs | 12 +- .../proto/src/logical_plan/from_proto.rs | 3 - datafusion/proto/src/logical_plan/to_proto.rs | 6 - .../proto/src/physical_plan/to_proto.rs | 19 +- .../tests/cases/roundtrip_logical_plan.rs | 4 + 15 files changed, 481 insertions(+), 856 deletions(-) create mode 100644 datafusion/functions-aggregate/src/bit_and_or_xor.rs delete mode 100644 datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 441e8953dffc..a7fbf26febb1 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -47,12 +47,6 @@ pub enum AggregateFunction { Correlation, /// Grouping Grouping, - /// Bit And - BitAnd, - /// Bit Or - BitOr, - /// Bit Xor - BitXor, /// Bool And BoolAnd, /// Bool Or @@ -72,9 +66,6 @@ impl AggregateFunction { NthValue => "NTH_VALUE", Correlation => "CORR", Grouping => "GROUPING", - BitAnd => "BIT_AND", - BitOr => "BIT_OR", - BitXor => "BIT_XOR", BoolAnd => "BOOL_AND", BoolOr => "BOOL_OR", StringAgg => "STRING_AGG", @@ -94,9 +85,6 @@ impl FromStr for AggregateFunction { Ok(match name { // general "avg" => AggregateFunction::Avg, - "bit_and" => AggregateFunction::BitAnd, - "bit_or" => AggregateFunction::BitOr, - "bit_xor" => AggregateFunction::BitXor, "bool_and" => AggregateFunction::BoolAnd, "bool_or" => AggregateFunction::BoolOr, "max" => AggregateFunction::Max, @@ -144,9 +132,6 @@ impl AggregateFunction { // The coerced_data_types is same with input_types. Ok(coerced_data_types[0].clone()) } - AggregateFunction::BitAnd - | AggregateFunction::BitOr - | AggregateFunction::BitXor => Ok(coerced_data_types[0].clone()), AggregateFunction::BoolAnd | AggregateFunction::BoolOr => { Ok(DataType::Boolean) } @@ -199,11 +184,6 @@ impl AggregateFunction { .collect::>(); Signature::uniform(1, valid, Volatility::Immutable) } - AggregateFunction::BitAnd - | AggregateFunction::BitOr - | AggregateFunction::BitXor => { - Signature::uniform(1, INTEGERS.to_vec(), Volatility::Immutable) - } AggregateFunction::BoolAnd | AggregateFunction::BoolOr => { Signature::uniform(1, vec![DataType::Boolean], Volatility::Immutable) } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 98324ed6120b..a216c98899fe 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -121,20 +121,6 @@ pub fn coerce_types( }; Ok(vec![v]) } - AggregateFunction::BitAnd - | AggregateFunction::BitOr - | AggregateFunction::BitXor => { - // Refer to https://www.postgresql.org/docs/8.2/functions-aggregate.html doc - // smallint, int, bigint, real, double precision, decimal, or interval. - if !is_bit_and_or_xor_support_arg_type(&input_types[0]) { - return plan_err!( - "The function {:?} does not support inputs of type {:?}.", - agg_fun, - input_types[0] - ); - } - Ok(input_types.to_vec()) - } AggregateFunction::BoolAnd | AggregateFunction::BoolOr => { // Refer to https://www.postgresql.org/docs/8.2/functions-aggregate.html doc // smallint, int, bigint, real, double precision, decimal, or interval. @@ -350,10 +336,6 @@ pub fn avg_sum_type(arg_type: &DataType) -> Result { } } -pub fn is_bit_and_or_xor_support_arg_type(arg_type: &DataType) -> bool { - NUMERICS.contains(arg_type) -} - pub fn is_bool_and_or_support_arg_type(arg_type: &DataType) -> bool { matches!(arg_type, DataType::Boolean) } diff --git a/datafusion/functions-aggregate/src/bit_and_or_xor.rs b/datafusion/functions-aggregate/src/bit_and_or_xor.rs new file mode 100644 index 000000000000..19e24f547d8a --- /dev/null +++ b/datafusion/functions-aggregate/src/bit_and_or_xor.rs @@ -0,0 +1,458 @@ +// 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. + +//! Defines `BitAnd`, `BitOr`, `BitXor` and `BitXor DISTINCT` aggregate accumulators + +use std::any::Any; +use std::collections::HashSet; +use std::fmt::{Display, Formatter}; + +use ahash::RandomState; +use arrow::array::{downcast_integer, Array, ArrayRef, AsArray}; +use arrow::datatypes::{ + ArrowNativeType, ArrowNumericType, DataType, Int16Type, Int32Type, Int64Type, + Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, +}; +use arrow_schema::Field; + +use datafusion_common::cast::as_list_array; +use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::type_coercion::aggregates::INTEGERS; +use datafusion_expr::utils::format_state_name; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, +}; + +use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; +use std::ops::{BitAndAssign, BitOrAssign, BitXorAssign}; + +/// This macro helps create group accumulators based on bitwise operations typically used internally +/// and might not be necessary for users to call directly. +macro_rules! group_accumulator_helper { + ($t:ty, $dt:expr, $opr:expr) => { + match $opr { + BitwiseOperationType::And => Ok(Box::new( + PrimitiveGroupsAccumulator::<$t, _>::new($dt, |x, y| x.bitand_assign(y)) + .with_starting_value(!0), + )), + BitwiseOperationType::Or => Ok(Box::new( + PrimitiveGroupsAccumulator::<$t, _>::new($dt, |x, y| x.bitor_assign(y)), + )), + BitwiseOperationType::Xor => Ok(Box::new( + PrimitiveGroupsAccumulator::<$t, _>::new($dt, |x, y| x.bitxor_assign(y)), + )), + } + }; +} + +/// `accumulator_helper` is a macro accepting (ArrowPrimitiveType, BitwiseOperationType, bool) +macro_rules! accumulator_helper { + ($t:ty, $opr:expr, $is_distinct: expr) => { + match $opr { + BitwiseOperationType::And => Ok(Box::>::default()), + BitwiseOperationType::Or => Ok(Box::>::default()), + BitwiseOperationType::Xor => { + if $is_distinct { + Ok(Box::>::default()) + } else { + Ok(Box::>::default()) + } + } + } + }; +} + +/// AND, OR and XOR only supports a subset of numeric types +/// +/// `args` is [AccumulatorArgs] +/// `opr` is [BitwiseOperationType] +/// `is_distinct` is boolean value indicating whether the operation is distinct or not. +macro_rules! downcast_bitwise_accumulator { + ($args:ident, $opr:expr, $is_distinct: expr) => { + match $args.data_type { + DataType::Int8 => accumulator_helper!(Int8Type, $opr, $is_distinct), + DataType::Int16 => accumulator_helper!(Int16Type, $opr, $is_distinct), + DataType::Int32 => accumulator_helper!(Int32Type, $opr, $is_distinct), + DataType::Int64 => accumulator_helper!(Int64Type, $opr, $is_distinct), + DataType::UInt8 => accumulator_helper!(UInt8Type, $opr, $is_distinct), + DataType::UInt16 => accumulator_helper!(UInt16Type, $opr, $is_distinct), + DataType::UInt32 => accumulator_helper!(UInt32Type, $opr, $is_distinct), + DataType::UInt64 => accumulator_helper!(UInt64Type, $opr, $is_distinct), + _ => { + not_impl_err!( + "{} not supported for {}: {}", + stringify!($opr), + $args.name, + $args.data_type + ) + } + } + }; +} + +/// Simplifies the creation of User-Defined Aggregate Functions (UDAFs) for performing bitwise operations in a declarative manner. +/// +/// `EXPR_FN` identifier used to name the generated expression function. +/// `AGGREGATE_UDF_FN` is an identifier used to name the underlying UDAF function. +/// `OPR_TYPE` is an expression that evaluates to the type of bitwise operation to be performed. +macro_rules! make_bitwise_udaf_expr_and_func { + ($EXPR_FN:ident, $AGGREGATE_UDF_FN:ident, $OPR_TYPE:expr) => { + make_udaf_expr!( + $EXPR_FN, + expr_x, + concat!( + "Returns the bitwise", + stringify!($OPR_TYPE), + "of a group of values" + ), + $AGGREGATE_UDF_FN + ); + create_func!( + $EXPR_FN, + $AGGREGATE_UDF_FN, + BitwiseOperation::new($OPR_TYPE, stringify!($EXPR_FN)) + ); + }; +} + +make_bitwise_udaf_expr_and_func!(bit_and, bit_and_udaf, BitwiseOperationType::And); +make_bitwise_udaf_expr_and_func!(bit_or, bit_or_udaf, BitwiseOperationType::Or); +make_bitwise_udaf_expr_and_func!(bit_xor, bit_xor_udaf, BitwiseOperationType::Xor); + +/// The different types of bitwise operations that can be performed. +#[derive(Debug, Clone, Eq, PartialEq)] +enum BitwiseOperationType { + And, + Or, + Xor, +} + +impl Display for BitwiseOperationType { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{:?}", self) + } +} + +/// [BitwiseOperation] struct encapsulates information about a bitwise operation. +#[derive(Debug)] +struct BitwiseOperation { + signature: Signature, + /// `operation` indicates the type of bitwise operation to be performed. + operation: BitwiseOperationType, + func_name: &'static str, +} + +impl BitwiseOperation { + pub fn new(operator: BitwiseOperationType, func_name: &'static str) -> Self { + Self { + operation: operator, + signature: Signature::uniform(1, INTEGERS.to_vec(), Volatility::Immutable), + func_name, + } + } +} + +impl AggregateUDFImpl for BitwiseOperation { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + self.func_name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + let arg_type = &arg_types[0]; + if !arg_type.is_integer() { + return exec_err!( + "[return_type] {} not supported for {}", + self.name(), + arg_type + ); + } + Ok(arg_type.clone()) + } + + fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { + downcast_bitwise_accumulator!(acc_args, self.operation, acc_args.is_distinct) + } + + fn state_fields(&self, args: StateFieldsArgs) -> Result> { + if self.operation == BitwiseOperationType::Xor && args.is_distinct { + Ok(vec![Field::new_list( + format_state_name( + args.name, + format!("{} distinct", self.name()).as_str(), + ), + Field::new("item", args.return_type.clone(), true), + false, + )]) + } else { + Ok(vec![Field::new( + format_state_name(args.name, self.name()), + args.return_type.clone(), + true, + )]) + } + } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + let data_type = args.data_type; + let operation = &self.operation; + downcast_integer! { + data_type => (group_accumulator_helper, data_type, operation), + _ => not_impl_err!( + "GroupsAccumulator not supported for {} with {}", + self.name(), + data_type + ), + } + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } +} + +struct BitAndAccumulator { + value: Option, +} + +impl std::fmt::Debug for BitAndAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "BitAndAccumulator({})", T::DATA_TYPE) + } +} + +impl Default for BitAndAccumulator { + fn default() -> Self { + Self { value: None } + } +} + +impl Accumulator for BitAndAccumulator +where + T::Native: std::ops::BitAnd, +{ + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if let Some(x) = arrow::compute::bit_and(values[0].as_primitive::()) { + let v = self.value.get_or_insert(x); + *v = *v & x; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![self.evaluate()?]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + self.update_batch(states) + } +} + +struct BitOrAccumulator { + value: Option, +} + +impl std::fmt::Debug for BitOrAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "BitOrAccumulator({})", T::DATA_TYPE) + } +} + +impl Default for BitOrAccumulator { + fn default() -> Self { + Self { value: None } + } +} + +impl Accumulator for BitOrAccumulator +where + T::Native: std::ops::BitOr, +{ + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if let Some(x) = arrow::compute::bit_or(values[0].as_primitive::()) { + let v = self.value.get_or_insert(T::Native::usize_as(0)); + *v = *v | x; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![self.evaluate()?]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + self.update_batch(states) + } +} + +struct BitXorAccumulator { + value: Option, +} + +impl std::fmt::Debug for BitXorAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "BitXorAccumulator({})", T::DATA_TYPE) + } +} + +impl Default for BitXorAccumulator { + fn default() -> Self { + Self { value: None } + } +} + +impl Accumulator for BitXorAccumulator +where + T::Native: std::ops::BitXor, +{ + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if let Some(x) = arrow::compute::bit_xor(values[0].as_primitive::()) { + let v = self.value.get_or_insert(T::Native::usize_as(0)); + *v = *v ^ x; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![self.evaluate()?]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + self.update_batch(states) + } +} + +struct DistinctBitXorAccumulator { + values: HashSet, +} + +impl std::fmt::Debug for DistinctBitXorAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "DistinctBitXorAccumulator({})", T::DATA_TYPE) + } +} + +impl Default for DistinctBitXorAccumulator { + fn default() -> Self { + Self { + values: HashSet::default(), + } + } +} + +impl Accumulator for DistinctBitXorAccumulator +where + T::Native: std::ops::BitXor + std::hash::Hash + Eq, +{ + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if values.is_empty() { + return Ok(()); + } + + let array = values[0].as_primitive::(); + match array.nulls().filter(|x| x.null_count() > 0) { + Some(n) => { + for idx in n.valid_indices() { + self.values.insert(array.value(idx)); + } + } + None => array.values().iter().for_each(|x| { + self.values.insert(*x); + }), + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + let mut acc = T::Native::usize_as(0); + for distinct_value in self.values.iter() { + acc = acc ^ *distinct_value; + } + let v = (!self.values.is_empty()).then_some(acc); + ScalarValue::new_primitive::(v, &T::DATA_TYPE) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + + self.values.capacity() * std::mem::size_of::() + } + + fn state(&mut self) -> Result> { + // 1. Stores aggregate state in `ScalarValue::List` + // 2. Constructs `ScalarValue::List` state from distinct numeric stored in hash set + let state_out = { + let values = self + .values + .iter() + .map(|x| ScalarValue::new_primitive::(Some(*x), &T::DATA_TYPE)) + .collect::>>()?; + + let arr = ScalarValue::new_list(&values, &T::DATA_TYPE); + vec![ScalarValue::List(arr)] + }; + Ok(state_out) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + if let Some(state) = states.first() { + let list_arr = as_list_array(state)?; + for arr in list_arr.iter().flatten() { + self.update_batch(&[arr])?; + } + } + Ok(()) + } +} diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index daddb9d93f78..990303bd1de3 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -69,6 +69,7 @@ pub mod variance; pub mod approx_median; pub mod approx_percentile_cont; pub mod approx_percentile_cont_with_weight; +pub mod bit_and_or_xor; use crate::approx_percentile_cont::approx_percentile_cont_udaf; use crate::approx_percentile_cont_with_weight::approx_percentile_cont_with_weight_udaf; @@ -84,6 +85,9 @@ pub mod expr_fn { pub use super::approx_median::approx_median; pub use super::approx_percentile_cont::approx_percentile_cont; pub use super::approx_percentile_cont_with_weight::approx_percentile_cont_with_weight; + pub use super::bit_and_or_xor::bit_and; + pub use super::bit_and_or_xor::bit_or; + pub use super::bit_and_or_xor::bit_xor; pub use super::count::count; pub use super::count::count_distinct; pub use super::covariance::covar_pop; @@ -134,6 +138,9 @@ pub fn all_default_aggregate_functions() -> Vec> { approx_distinct::approx_distinct_udaf(), approx_percentile_cont_udaf(), approx_percentile_cont_with_weight_udaf(), + bit_and_or_xor::bit_and_udaf(), + bit_and_or_xor::bit_or_udaf(), + bit_and_or_xor::bit_xor_udaf(), ] } diff --git a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs deleted file mode 100644 index 3fa225c5e479..000000000000 --- a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs +++ /dev/null @@ -1,695 +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. - -//! Defines BitAnd, BitOr, and BitXor Aggregate accumulators - -use ahash::RandomState; -use datafusion_common::cast::as_list_array; -use std::any::Any; -use std::sync::Arc; - -use crate::{AggregateExpr, PhysicalExpr}; -use arrow::datatypes::DataType; -use arrow::{array::ArrayRef, datatypes::Field}; -use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; -use datafusion_expr::{Accumulator, GroupsAccumulator}; -use std::collections::HashSet; - -use crate::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; -use crate::aggregate::utils::down_cast_any_ref; -use crate::expressions::format_state_name; -use arrow::array::Array; -use arrow::compute::{bit_and, bit_or, bit_xor}; -use arrow_array::cast::AsArray; -use arrow_array::{downcast_integer, ArrowNumericType}; -use arrow_buffer::ArrowNativeType; - -/// BIT_AND aggregate expression -#[derive(Debug, Clone)] -pub struct BitAnd { - name: String, - pub data_type: DataType, - expr: Arc, - nullable: bool, -} - -impl BitAnd { - /// Create a new BIT_AND aggregate function - pub fn new( - expr: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - nullable: true, - } - } -} - -impl AggregateExpr for BitAnd { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - macro_rules! helper { - ($t:ty) => { - Ok(Box::>::default()) - }; - } - downcast_integer! { - &self.data_type => (helper), - _ => Err(DataFusionError::NotImplemented(format!( - "BitAndAccumulator not supported for {} with {}", - self.name(), - self.data_type - ))), - } - } - - fn state_fields(&self) -> Result> { - Ok(vec![Field::new( - format_state_name(&self.name, "bit_and"), - self.data_type.clone(), - self.nullable, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { - true - } - - fn create_groups_accumulator(&self) -> Result> { - use std::ops::BitAndAssign; - - // Note the default value for BitAnd should be all set, i.e. `!0` - macro_rules! helper { - ($t:ty, $dt:expr) => { - Ok(Box::new( - PrimitiveGroupsAccumulator::<$t, _>::new($dt, |x, y| { - x.bitand_assign(y) - }) - .with_starting_value(!0), - )) - }; - } - - let data_type = &self.data_type; - downcast_integer! { - data_type => (helper, data_type), - _ => not_impl_err!( - "GroupsAccumulator not supported for {} with {}", - self.name(), - self.data_type - ), - } - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone())) - } -} - -impl PartialEq for BitAnd { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.nullable == x.nullable - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - -struct BitAndAccumulator { - value: Option, -} - -impl std::fmt::Debug for BitAndAccumulator { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "BitAndAccumulator({})", T::DATA_TYPE) - } -} - -impl Default for BitAndAccumulator { - fn default() -> Self { - Self { value: None } - } -} - -impl Accumulator for BitAndAccumulator -where - T::Native: std::ops::BitAnd, -{ - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if let Some(x) = bit_and(values[0].as_primitive::()) { - let v = self.value.get_or_insert(x); - *v = *v & x; - } - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - self.update_batch(states) - } - - fn state(&mut self) -> Result> { - Ok(vec![self.evaluate()?]) - } - - fn evaluate(&mut self) -> Result { - ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - } -} - -/// BIT_OR aggregate expression -#[derive(Debug, Clone)] -pub struct BitOr { - name: String, - pub data_type: DataType, - expr: Arc, - nullable: bool, -} - -impl BitOr { - /// Create a new BIT_OR aggregate function - pub fn new( - expr: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - nullable: true, - } - } -} - -impl AggregateExpr for BitOr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - macro_rules! helper { - ($t:ty) => { - Ok(Box::>::default()) - }; - } - downcast_integer! { - &self.data_type => (helper), - _ => Err(DataFusionError::NotImplemented(format!( - "BitOrAccumulator not supported for {} with {}", - self.name(), - self.data_type - ))), - } - } - - fn state_fields(&self) -> Result> { - Ok(vec![Field::new( - format_state_name(&self.name, "bit_or"), - self.data_type.clone(), - self.nullable, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { - true - } - - fn create_groups_accumulator(&self) -> Result> { - use std::ops::BitOrAssign; - macro_rules! helper { - ($t:ty, $dt:expr) => { - Ok(Box::new(PrimitiveGroupsAccumulator::<$t, _>::new( - $dt, - |x, y| x.bitor_assign(y), - ))) - }; - } - - let data_type = &self.data_type; - downcast_integer! { - data_type => (helper, data_type), - _ => not_impl_err!( - "GroupsAccumulator not supported for {} with {}", - self.name(), - self.data_type - ), - } - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone())) - } -} - -impl PartialEq for BitOr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.nullable == x.nullable - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - -struct BitOrAccumulator { - value: Option, -} - -impl std::fmt::Debug for BitOrAccumulator { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "BitOrAccumulator({})", T::DATA_TYPE) - } -} - -impl Default for BitOrAccumulator { - fn default() -> Self { - Self { value: None } - } -} - -impl Accumulator for BitOrAccumulator -where - T::Native: std::ops::BitOr, -{ - fn state(&mut self) -> Result> { - Ok(vec![self.evaluate()?]) - } - - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if let Some(x) = bit_or(values[0].as_primitive::()) { - let v = self.value.get_or_insert(T::Native::usize_as(0)); - *v = *v | x; - } - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - self.update_batch(states) - } - - fn evaluate(&mut self) -> Result { - ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - } -} - -/// BIT_XOR aggregate expression -#[derive(Debug, Clone)] -pub struct BitXor { - name: String, - pub data_type: DataType, - expr: Arc, - nullable: bool, -} - -impl BitXor { - /// Create a new BIT_XOR aggregate function - pub fn new( - expr: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - nullable: true, - } - } -} - -impl AggregateExpr for BitXor { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - macro_rules! helper { - ($t:ty) => { - Ok(Box::>::default()) - }; - } - downcast_integer! { - &self.data_type => (helper), - _ => Err(DataFusionError::NotImplemented(format!( - "BitXor not supported for {} with {}", - self.name(), - self.data_type - ))), - } - } - - fn state_fields(&self) -> Result> { - Ok(vec![Field::new( - format_state_name(&self.name, "bit_xor"), - self.data_type.clone(), - self.nullable, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { - true - } - - fn create_groups_accumulator(&self) -> Result> { - use std::ops::BitXorAssign; - macro_rules! helper { - ($t:ty, $dt:expr) => { - Ok(Box::new(PrimitiveGroupsAccumulator::<$t, _>::new( - $dt, - |x, y| x.bitxor_assign(y), - ))) - }; - } - - let data_type = &self.data_type; - downcast_integer! { - data_type => (helper, data_type), - _ => not_impl_err!( - "GroupsAccumulator not supported for {} with {}", - self.name(), - self.data_type - ), - } - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone())) - } -} - -impl PartialEq for BitXor { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.nullable == x.nullable - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - -struct BitXorAccumulator { - value: Option, -} - -impl std::fmt::Debug for BitXorAccumulator { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "BitXorAccumulator({})", T::DATA_TYPE) - } -} - -impl Default for BitXorAccumulator { - fn default() -> Self { - Self { value: None } - } -} - -impl Accumulator for BitXorAccumulator -where - T::Native: std::ops::BitXor, -{ - fn state(&mut self) -> Result> { - Ok(vec![self.evaluate()?]) - } - - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if let Some(x) = bit_xor(values[0].as_primitive::()) { - let v = self.value.get_or_insert(T::Native::usize_as(0)); - *v = *v ^ x; - } - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - self.update_batch(states) - } - - fn evaluate(&mut self) -> Result { - ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - } -} - -/// Expression for a BIT_XOR(DISTINCT) aggregation. -#[derive(Debug, Clone)] -pub struct DistinctBitXor { - name: String, - pub data_type: DataType, - expr: Arc, - nullable: bool, -} - -impl DistinctBitXor { - /// Create a new DistinctBitXor aggregate function - pub fn new( - expr: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - nullable: true, - } - } -} - -impl AggregateExpr for DistinctBitXor { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - macro_rules! helper { - ($t:ty) => { - Ok(Box::>::default()) - }; - } - downcast_integer! { - &self.data_type => (helper), - _ => Err(DataFusionError::NotImplemented(format!( - "DistinctBitXorAccumulator not supported for {} with {}", - self.name(), - self.data_type - ))), - } - } - - fn state_fields(&self) -> Result> { - // State field is a List which stores items to rebuild hash set. - Ok(vec![Field::new_list( - format_state_name(&self.name, "bit_xor distinct"), - Field::new("item", self.data_type.clone(), true), - false, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn name(&self) -> &str { - &self.name - } -} - -impl PartialEq for DistinctBitXor { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.nullable == x.nullable - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - -struct DistinctBitXorAccumulator { - values: HashSet, -} - -impl std::fmt::Debug for DistinctBitXorAccumulator { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "DistinctBitXorAccumulator({})", T::DATA_TYPE) - } -} - -impl Default for DistinctBitXorAccumulator { - fn default() -> Self { - Self { - values: HashSet::default(), - } - } -} - -impl Accumulator for DistinctBitXorAccumulator -where - T::Native: std::ops::BitXor + std::hash::Hash + Eq, -{ - fn state(&mut self) -> Result> { - // 1. Stores aggregate state in `ScalarValue::List` - // 2. Constructs `ScalarValue::List` state from distinct numeric stored in hash set - let state_out = { - let values = self - .values - .iter() - .map(|x| ScalarValue::new_primitive::(Some(*x), &T::DATA_TYPE)) - .collect::>>()?; - - let arr = ScalarValue::new_list(&values, &T::DATA_TYPE); - vec![ScalarValue::List(arr)] - }; - Ok(state_out) - } - - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if values.is_empty() { - return Ok(()); - } - - let array = values[0].as_primitive::(); - match array.nulls().filter(|x| x.null_count() > 0) { - Some(n) => { - for idx in n.valid_indices() { - self.values.insert(array.value(idx)); - } - } - None => array.values().iter().for_each(|x| { - self.values.insert(*x); - }), - } - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - if let Some(state) = states.first() { - let list_arr = as_list_array(state)?; - for arr in list_arr.iter().flatten() { - self.update_batch(&[arr])?; - } - } - Ok(()) - } - - fn evaluate(&mut self) -> Result { - let mut acc = T::Native::usize_as(0); - for distinct_value in self.values.iter() { - acc = acc ^ *distinct_value; - } - let v = (!self.values.is_empty()).then_some(acc); - ScalarValue::new_primitive::(v, &T::DATA_TYPE) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - + self.values.capacity() * std::mem::size_of::() - } -} diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index a1f5f153a9ff..6c01decdbf95 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -66,26 +66,6 @@ pub fn create_aggregate_expr( name, data_type, )), - (AggregateFunction::BitAnd, _) => Arc::new(expressions::BitAnd::new( - input_phy_exprs[0].clone(), - name, - data_type, - )), - (AggregateFunction::BitOr, _) => Arc::new(expressions::BitOr::new( - input_phy_exprs[0].clone(), - name, - data_type, - )), - (AggregateFunction::BitXor, false) => Arc::new(expressions::BitXor::new( - input_phy_exprs[0].clone(), - name, - data_type, - )), - (AggregateFunction::BitXor, true) => Arc::new(expressions::DistinctBitXor::new( - input_phy_exprs[0].clone(), - name, - data_type, - )), (AggregateFunction::BoolAnd, _) => Arc::new(expressions::BoolAnd::new( input_phy_exprs[0].clone(), name, @@ -202,12 +182,10 @@ mod tests { use datafusion_expr::{type_coercion, Signature}; use crate::expressions::{ - try_cast, ArrayAgg, Avg, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, - DistinctArrayAgg, Max, Min, + try_cast, ArrayAgg, Avg, BoolAnd, BoolOr, DistinctArrayAgg, Max, Min, }; use super::*; - #[test] fn test_approx_expr() -> Result<()> { let funcs = vec![AggregateFunction::ArrayAgg]; @@ -319,60 +297,6 @@ mod tests { Ok(()) } - #[test] - fn test_bit_and_or_xor_expr() -> Result<()> { - let funcs = vec![ - AggregateFunction::BitAnd, - AggregateFunction::BitOr, - AggregateFunction::BitXor, - ]; - let data_types = vec![DataType::UInt64, DataType::Int64]; - for fun in funcs { - for data_type in &data_types { - let input_schema = - Schema::new(vec![Field::new("c1", data_type.clone(), true)]); - let input_phy_exprs: Vec> = vec![Arc::new( - expressions::Column::new_with_schema("c1", &input_schema).unwrap(), - )]; - let result_agg_phy_exprs = create_physical_agg_expr_for_test( - &fun, - false, - &input_phy_exprs[0..1], - &input_schema, - "c1", - )?; - match fun { - AggregateFunction::BitAnd => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - assert_eq!( - Field::new("c1", data_type.clone(), true), - result_agg_phy_exprs.field().unwrap() - ); - } - AggregateFunction::BitOr => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - assert_eq!( - Field::new("c1", data_type.clone(), true), - result_agg_phy_exprs.field().unwrap() - ); - } - AggregateFunction::BitXor => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - assert_eq!( - Field::new("c1", data_type.clone(), true), - result_agg_phy_exprs.field().unwrap() - ); - } - _ => {} - }; - } - } - Ok(()) - } - #[test] fn test_bool_and_or_expr() -> Result<()> { let funcs = vec![AggregateFunction::BoolAnd, AggregateFunction::BoolOr]; diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index c20902c11b86..0b1f5f577435 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -21,7 +21,6 @@ pub(crate) mod array_agg; pub(crate) mod array_agg_distinct; pub(crate) mod array_agg_ordered; pub(crate) mod average; -pub(crate) mod bit_and_or_xor; pub(crate) mod bool_and_or; pub(crate) mod correlation; pub(crate) mod covariance; diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index b9a159b21e3d..bffaafd7dac2 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -40,7 +40,6 @@ pub use crate::aggregate::array_agg_distinct::DistinctArrayAgg; pub use crate::aggregate::array_agg_ordered::OrderSensitiveArrayAgg; pub use crate::aggregate::average::Avg; pub use crate::aggregate::average::AvgAccumulator; -pub use crate::aggregate::bit_and_or_xor::{BitAnd, BitOr, BitXor, DistinctBitXor}; pub use crate::aggregate::bool_and_or::{BoolAnd, BoolOr}; pub use crate::aggregate::build_in::create_aggregate_expr; pub use crate::aggregate::correlation::Correlation; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index e5578ae62f3e..ae4445eaa8ce 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -491,9 +491,9 @@ enum AggregateFunction { // APPROX_PERCENTILE_CONT_WITH_WEIGHT = 16; GROUPING = 17; // MEDIAN = 18; - BIT_AND = 19; - BIT_OR = 20; - BIT_XOR = 21; + // BIT_AND = 19; + // BIT_OR = 20; + // BIT_XOR = 21; BOOL_AND = 22; BOOL_OR = 23; // REGR_SLOPE = 26; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 4a7b9610e5bc..243c75435f8d 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -538,9 +538,6 @@ impl serde::Serialize for AggregateFunction { Self::ArrayAgg => "ARRAY_AGG", Self::Correlation => "CORRELATION", Self::Grouping => "GROUPING", - Self::BitAnd => "BIT_AND", - Self::BitOr => "BIT_OR", - Self::BitXor => "BIT_XOR", Self::BoolAnd => "BOOL_AND", Self::BoolOr => "BOOL_OR", Self::StringAgg => "STRING_AGG", @@ -562,9 +559,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "ARRAY_AGG", "CORRELATION", "GROUPING", - "BIT_AND", - "BIT_OR", - "BIT_XOR", "BOOL_AND", "BOOL_OR", "STRING_AGG", @@ -615,9 +609,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "ARRAY_AGG" => Ok(AggregateFunction::ArrayAgg), "CORRELATION" => Ok(AggregateFunction::Correlation), "GROUPING" => Ok(AggregateFunction::Grouping), - "BIT_AND" => Ok(AggregateFunction::BitAnd), - "BIT_OR" => Ok(AggregateFunction::BitOr), - "BIT_XOR" => Ok(AggregateFunction::BitXor), "BOOL_AND" => Ok(AggregateFunction::BoolAnd), "BOOL_OR" => Ok(AggregateFunction::BoolOr), "STRING_AGG" => Ok(AggregateFunction::StringAgg), diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index ffaef445d668..1172eccb90fd 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1945,9 +1945,9 @@ pub enum AggregateFunction { /// APPROX_PERCENTILE_CONT_WITH_WEIGHT = 16; Grouping = 17, /// MEDIAN = 18; - BitAnd = 19, - BitOr = 20, - BitXor = 21, + /// BIT_AND = 19; + /// BIT_OR = 20; + /// BIT_XOR = 21; BoolAnd = 22, BoolOr = 23, /// REGR_SLOPE = 26; @@ -1975,9 +1975,6 @@ impl AggregateFunction { AggregateFunction::ArrayAgg => "ARRAY_AGG", AggregateFunction::Correlation => "CORRELATION", AggregateFunction::Grouping => "GROUPING", - AggregateFunction::BitAnd => "BIT_AND", - AggregateFunction::BitOr => "BIT_OR", - AggregateFunction::BitXor => "BIT_XOR", AggregateFunction::BoolAnd => "BOOL_AND", AggregateFunction::BoolOr => "BOOL_OR", AggregateFunction::StringAgg => "STRING_AGG", @@ -1993,9 +1990,6 @@ impl AggregateFunction { "ARRAY_AGG" => Some(Self::ArrayAgg), "CORRELATION" => Some(Self::Correlation), "GROUPING" => Some(Self::Grouping), - "BIT_AND" => Some(Self::BitAnd), - "BIT_OR" => Some(Self::BitOr), - "BIT_XOR" => Some(Self::BitXor), "BOOL_AND" => Some(Self::BoolAnd), "BOOL_OR" => Some(Self::BoolOr), "STRING_AGG" => Some(Self::StringAgg), diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 25b7413a984a..43cc352f98dd 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -140,9 +140,6 @@ impl From for AggregateFunction { protobuf::AggregateFunction::Min => Self::Min, protobuf::AggregateFunction::Max => Self::Max, protobuf::AggregateFunction::Avg => Self::Avg, - protobuf::AggregateFunction::BitAnd => Self::BitAnd, - protobuf::AggregateFunction::BitOr => Self::BitOr, - protobuf::AggregateFunction::BitXor => Self::BitXor, protobuf::AggregateFunction::BoolAnd => Self::BoolAnd, protobuf::AggregateFunction::BoolOr => Self::BoolOr, protobuf::AggregateFunction::ArrayAgg => Self::ArrayAgg, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index d9548325dac3..33a58daeaf0a 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -111,9 +111,6 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::Min => Self::Min, AggregateFunction::Max => Self::Max, AggregateFunction::Avg => Self::Avg, - AggregateFunction::BitAnd => Self::BitAnd, - AggregateFunction::BitOr => Self::BitOr, - AggregateFunction::BitXor => Self::BitXor, AggregateFunction::BoolAnd => Self::BoolAnd, AggregateFunction::BoolOr => Self::BoolOr, AggregateFunction::ArrayAgg => Self::ArrayAgg, @@ -380,9 +377,6 @@ pub fn serialize_expr( AggregateFunction::ArrayAgg => protobuf::AggregateFunction::ArrayAgg, AggregateFunction::Min => protobuf::AggregateFunction::Min, AggregateFunction::Max => protobuf::AggregateFunction::Max, - AggregateFunction::BitAnd => protobuf::AggregateFunction::BitAnd, - AggregateFunction::BitOr => protobuf::AggregateFunction::BitOr, - AggregateFunction::BitXor => protobuf::AggregateFunction::BitXor, AggregateFunction::BoolAnd => protobuf::AggregateFunction::BoolAnd, AggregateFunction::BoolOr => protobuf::AggregateFunction::BoolOr, AggregateFunction::Avg => protobuf::AggregateFunction::Avg, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 3a4c35a93e16..886179bf5627 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -23,11 +23,11 @@ use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ - ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, - CastExpr, Column, Correlation, CumeDist, DistinctArrayAgg, DistinctBitXor, Grouping, - InListExpr, IsNotNullExpr, IsNullExpr, Literal, Max, Min, NegativeExpr, NotExpr, - NthValue, NthValueAgg, Ntile, OrderSensitiveArrayAgg, Rank, RankType, RowNumber, - StringAgg, TryCastExpr, WindowShift, + ArrayAgg, Avg, BinaryExpr, BoolAnd, BoolOr, CaseExpr, CastExpr, Column, Correlation, + CumeDist, DistinctArrayAgg, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, + Max, Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, + OrderSensitiveArrayAgg, Rank, RankType, RowNumber, StringAgg, TryCastExpr, + WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; @@ -241,15 +241,6 @@ fn aggr_expr_to_aggr_fn(expr: &dyn AggregateExpr) -> Result { let inner = if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::Grouping - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::BitAnd - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::BitOr - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::BitXor - } else if aggr_expr.downcast_ref::().is_some() { - distinct = true; - protobuf::AggregateFunction::BitXor } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::BoolAnd } else if aggr_expr.downcast_ref::().is_some() { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index a496e226855a..52696a106183 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -59,6 +59,7 @@ use datafusion_expr::{ TryCast, Volatility, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, WindowUDF, WindowUDFImpl, }; +use datafusion_functions_aggregate::expr_fn::{bit_and, bit_or, bit_xor}; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, logical_plan_to_bytes, logical_plan_to_bytes_with_extension_codec, @@ -665,6 +666,9 @@ async fn roundtrip_expr_api() -> Result<()> { approx_median(lit(2)), approx_percentile_cont(lit(2), lit(0.5)), approx_percentile_cont_with_weight(lit(2), lit(1), lit(0.5)), + bit_and(lit(2)), + bit_or(lit(2)), + bit_xor(lit(2)), ]; // ensure expressions created with the expr api can be round tripped From 9b1bb68e37688330fa47c73bdd733b206be8759e Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Mon, 17 Jun 2024 08:52:26 -0700 Subject: [PATCH 05/54] refactor: improve PoolType argument handling for CLI (#10940) * refactor: dont include fallback in match on mem_pool_type * refactor: improve PoolType argument handling --- datafusion-cli/src/lib.rs | 1 + datafusion-cli/src/main.rs | 29 +++++--------------- datafusion-cli/src/pool_type.rs | 48 +++++++++++++++++++++++++++++++++ 3 files changed, 55 insertions(+), 23 deletions(-) create mode 100644 datafusion-cli/src/pool_type.rs diff --git a/datafusion-cli/src/lib.rs b/datafusion-cli/src/lib.rs index 139a60b8cf16..5081436aa6c5 100644 --- a/datafusion-cli/src/lib.rs +++ b/datafusion-cli/src/lib.rs @@ -25,5 +25,6 @@ pub mod functions; pub mod helper; pub mod highlighter; pub mod object_storage; +pub mod pool_type; pub mod print_format; pub mod print_options; diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index f2b29fe78690..f469fda4f960 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -19,7 +19,6 @@ use std::collections::HashMap; use std::env; use std::path::Path; use std::process::ExitCode; -use std::str::FromStr; use std::sync::{Arc, OnceLock}; use datafusion::error::{DataFusionError, Result}; @@ -31,6 +30,7 @@ use datafusion_cli::catalog::DynamicFileCatalog; use datafusion_cli::functions::ParquetMetadataFunc; use datafusion_cli::{ exec, + pool_type::PoolType, print_format::PrintFormat, print_options::{MaxRows, PrintOptions}, DATAFUSION_CLI_VERSION, @@ -42,24 +42,6 @@ use mimalloc::MiMalloc; #[global_allocator] static GLOBAL: MiMalloc = MiMalloc; -#[derive(PartialEq, Debug)] -enum PoolType { - Greedy, - Fair, -} - -impl FromStr for PoolType { - type Err = String; - - fn from_str(s: &str) -> Result { - match s { - "Greedy" | "greedy" => Ok(PoolType::Greedy), - "Fair" | "fair" => Ok(PoolType::Fair), - _ => Err(format!("Invalid memory pool type '{}'", s)), - } - } -} - #[derive(Debug, Parser, PartialEq)] #[clap(author, version, about, long_about= None)] struct Args { @@ -127,9 +109,10 @@ struct Args { #[clap( long, - help = "Specify the memory pool type 'greedy' or 'fair', default to 'greedy'" + help = "Specify the memory pool type 'greedy' or 'fair'", + default_value_t = PoolType::Greedy )] - mem_pool_type: Option, + mem_pool_type: PoolType, #[clap( long, @@ -181,9 +164,9 @@ async fn main_inner() -> Result<()> { let memory_limit = extract_memory_pool_size(&memory_limit).unwrap(); // set memory pool type match args.mem_pool_type { - Some(PoolType::Fair) => rt_config + PoolType::Fair => rt_config .with_memory_pool(Arc::new(FairSpillPool::new(memory_limit))), - _ => rt_config + PoolType::Greedy => rt_config .with_memory_pool(Arc::new(GreedyMemoryPool::new(memory_limit))) } } else { diff --git a/datafusion-cli/src/pool_type.rs b/datafusion-cli/src/pool_type.rs new file mode 100644 index 000000000000..25763eba5c8c --- /dev/null +++ b/datafusion-cli/src/pool_type.rs @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::{ + fmt::{self, Display, Formatter}, + str::FromStr, +}; + +#[derive(PartialEq, Debug)] +pub enum PoolType { + Greedy, + Fair, +} + +impl FromStr for PoolType { + type Err = String; + + fn from_str(s: &str) -> Result { + match s { + "Greedy" | "greedy" => Ok(PoolType::Greedy), + "Fair" | "fair" => Ok(PoolType::Fair), + _ => Err(format!("Invalid memory pool type '{}'", s)), + } + } +} + +impl Display for PoolType { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + match self { + PoolType::Greedy => write!(f, "greedy"), + PoolType::Fair => write!(f, "fair"), + } + } +} From 861a2364bdf04854482384c29d9b64962da377fe Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Mon, 17 Jun 2024 10:16:20 -0700 Subject: [PATCH 06/54] feat: add CliSessionContext trait for cli (#10890) use CliSessionContext trait for cli --- .../examples/cli-session-context.rs | 97 ++++++++++++++++++ datafusion-cli/src/cli_context.rs | 98 +++++++++++++++++++ datafusion-cli/src/command.rs | 4 +- datafusion-cli/src/exec.rs | 28 +++--- datafusion-cli/src/lib.rs | 1 + datafusion-cli/src/object_storage.rs | 55 ++--------- 6 files changed, 220 insertions(+), 63 deletions(-) create mode 100644 datafusion-cli/examples/cli-session-context.rs create mode 100644 datafusion-cli/src/cli_context.rs diff --git a/datafusion-cli/examples/cli-session-context.rs b/datafusion-cli/examples/cli-session-context.rs new file mode 100644 index 000000000000..8da52ed84a5f --- /dev/null +++ b/datafusion-cli/examples/cli-session-context.rs @@ -0,0 +1,97 @@ +// 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. + +//! Shows an example of a custom session context that unions the input plan with itself. +//! To run this example, use `cargo run --example cli-session-context` from within the `datafusion-cli` directory. + +use std::sync::Arc; + +use datafusion::{ + dataframe::DataFrame, + error::DataFusionError, + execution::{context::SessionState, TaskContext}, + logical_expr::{LogicalPlan, LogicalPlanBuilder}, + prelude::SessionContext, +}; +use datafusion_cli::{ + cli_context::CliSessionContext, exec::exec_from_repl, print_options::PrintOptions, +}; +use object_store::ObjectStore; + +/// This is a toy example of a custom session context that unions the input plan with itself. +struct MyUnionerContext { + ctx: SessionContext, +} + +impl Default for MyUnionerContext { + fn default() -> Self { + Self { + ctx: SessionContext::new(), + } + } +} + +#[async_trait::async_trait] +impl CliSessionContext for MyUnionerContext { + fn task_ctx(&self) -> Arc { + self.ctx.task_ctx() + } + + fn session_state(&self) -> SessionState { + self.ctx.state() + } + + fn register_object_store( + &self, + url: &url::Url, + object_store: Arc, + ) -> Option> { + self.ctx.register_object_store(url, object_store) + } + + fn register_table_options_extension_from_scheme(&self, _scheme: &str) { + unimplemented!() + } + + async fn execute_logical_plan( + &self, + plan: LogicalPlan, + ) -> Result { + let new_plan = LogicalPlanBuilder::from(plan.clone()) + .union(plan.clone())? + .build()?; + + self.ctx.execute_logical_plan(new_plan).await + } +} + +#[tokio::main] +/// Runs the example. +pub async fn main() { + let mut my_ctx = MyUnionerContext::default(); + + let mut print_options = PrintOptions { + format: datafusion_cli::print_format::PrintFormat::Automatic, + quiet: false, + maxrows: datafusion_cli::print_options::MaxRows::Unlimited, + color: true, + }; + + exec_from_repl(&mut my_ctx, &mut print_options) + .await + .unwrap(); +} diff --git a/datafusion-cli/src/cli_context.rs b/datafusion-cli/src/cli_context.rs new file mode 100644 index 000000000000..516929ebacf1 --- /dev/null +++ b/datafusion-cli/src/cli_context.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 std::sync::Arc; + +use datafusion::{ + dataframe::DataFrame, + error::DataFusionError, + execution::{context::SessionState, TaskContext}, + logical_expr::LogicalPlan, + prelude::SessionContext, +}; +use object_store::ObjectStore; + +use crate::object_storage::{AwsOptions, GcpOptions}; + +#[async_trait::async_trait] +/// The CLI session context trait provides a way to have a session context that can be used with datafusion's CLI code. +pub trait CliSessionContext { + /// Get an atomic reference counted task context. + fn task_ctx(&self) -> Arc; + + /// Get the session state. + fn session_state(&self) -> SessionState; + + /// Register an object store with the session context. + fn register_object_store( + &self, + url: &url::Url, + object_store: Arc, + ) -> Option>; + + /// Register table options extension from scheme. + fn register_table_options_extension_from_scheme(&self, scheme: &str); + + /// Execute a logical plan and return a DataFrame. + async fn execute_logical_plan( + &self, + plan: LogicalPlan, + ) -> Result; +} + +#[async_trait::async_trait] +impl CliSessionContext for SessionContext { + fn task_ctx(&self) -> Arc { + self.task_ctx() + } + + fn session_state(&self) -> SessionState { + self.state() + } + + fn register_object_store( + &self, + url: &url::Url, + object_store: Arc, + ) -> Option> { + self.register_object_store(url, object_store) + } + + fn register_table_options_extension_from_scheme(&self, scheme: &str) { + match scheme { + // For Amazon S3 or Alibaba Cloud OSS + "s3" | "oss" | "cos" => { + // Register AWS specific table options in the session context: + self.register_table_options_extension(AwsOptions::default()) + } + // For Google Cloud Storage + "gs" | "gcs" => { + // Register GCP specific table options in the session context: + self.register_table_options_extension(GcpOptions::default()) + } + // For unsupported schemes, do nothing: + _ => {} + } + } + + async fn execute_logical_plan( + &self, + plan: LogicalPlan, + ) -> Result { + self.execute_logical_plan(plan).await + } +} diff --git a/datafusion-cli/src/command.rs b/datafusion-cli/src/command.rs index be6393351aed..1a6c023d3b50 100644 --- a/datafusion-cli/src/command.rs +++ b/datafusion-cli/src/command.rs @@ -17,6 +17,7 @@ //! Command within CLI +use crate::cli_context::CliSessionContext; use crate::exec::{exec_and_print, exec_from_lines}; use crate::functions::{display_all_functions, Function}; use crate::print_format::PrintFormat; @@ -28,7 +29,6 @@ use datafusion::arrow::record_batch::RecordBatch; use datafusion::common::exec_err; use datafusion::common::instant::Instant; use datafusion::error::{DataFusionError, Result}; -use datafusion::prelude::SessionContext; use std::fs::File; use std::io::BufReader; use std::str::FromStr; @@ -55,7 +55,7 @@ pub enum OutputFormat { impl Command { pub async fn execute( &self, - ctx: &mut SessionContext, + ctx: &mut dyn CliSessionContext, print_options: &mut PrintOptions, ) -> Result<()> { match self { diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 855d6a7cbbc9..c4c92be1525d 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -23,12 +23,13 @@ use std::io::prelude::*; use std::io::BufReader; use std::str::FromStr; +use crate::cli_context::CliSessionContext; use crate::helper::split_from_semicolon; use crate::print_format::PrintFormat; use crate::{ command::{Command, OutputFormat}, helper::{unescape_input, CliHelper}, - object_storage::{get_object_store, register_options}, + object_storage::get_object_store, print_options::{MaxRows, PrintOptions}, }; @@ -38,7 +39,6 @@ use datafusion::datasource::listing::ListingTableUrl; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::{DdlStatement, LogicalPlan}; use datafusion::physical_plan::{collect, execute_stream, ExecutionPlanProperties}; -use datafusion::prelude::SessionContext; use datafusion::sql::parser::{DFParser, Statement}; use datafusion::sql::sqlparser::dialect::dialect_from_str; @@ -50,7 +50,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 SessionContext, + ctx: &mut dyn CliSessionContext, commands: Vec, print_options: &PrintOptions, ) -> Result<()> { @@ -63,7 +63,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 SessionContext, + ctx: &mut dyn CliSessionContext, reader: &mut BufReader, print_options: &PrintOptions, ) -> Result<()> { @@ -103,7 +103,7 @@ pub async fn exec_from_lines( } pub async fn exec_from_files( - ctx: &mut SessionContext, + ctx: &mut dyn CliSessionContext, files: Vec, print_options: &PrintOptions, ) -> Result<()> { @@ -122,7 +122,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 SessionContext, + ctx: &mut dyn CliSessionContext, print_options: &mut PrintOptions, ) -> rustyline::Result<()> { let mut rl = Editor::new()?; @@ -205,7 +205,7 @@ pub async fn exec_from_repl( } pub(super) async fn exec_and_print( - ctx: &mut SessionContext, + ctx: &mut dyn CliSessionContext, print_options: &PrintOptions, sql: String, ) -> Result<()> { @@ -292,10 +292,10 @@ impl AdjustedPrintOptions { } async fn create_plan( - ctx: &mut SessionContext, + ctx: &mut dyn CliSessionContext, statement: Statement, ) -> Result { - let mut plan = ctx.state().statement_to_plan(statement).await?; + let mut plan = ctx.session_state().statement_to_plan(statement).await?; // Note that cmd is a mutable reference so that create_external_table function can remove all // datafusion-cli specific options before passing through to datafusion. Otherwise, datafusion @@ -354,7 +354,7 @@ async fn create_plan( /// alteration fails, or if the object store cannot be retrieved and registered /// successfully. pub(crate) async fn register_object_store_and_config_extensions( - ctx: &SessionContext, + ctx: &dyn CliSessionContext, location: &String, options: &HashMap, format: Option, @@ -369,17 +369,18 @@ pub(crate) async fn register_object_store_and_config_extensions( let url = table_path.as_ref(); // Register the options based on the scheme extracted from the location - register_options(ctx, scheme); + ctx.register_table_options_extension_from_scheme(scheme); // Clone and modify the default table options based on the provided options - let mut table_options = ctx.state().default_table_options().clone(); + let mut table_options = ctx.session_state().default_table_options().clone(); if let Some(format) = format { table_options.set_file_format(format); } table_options.alter_with_string_hash_map(options)?; // Retrieve the appropriate object store based on the scheme, URL, and modified table options - let store = get_object_store(&ctx.state(), scheme, url, &table_options).await?; + let store = + get_object_store(&ctx.session_state(), scheme, url, &table_options).await?; // Register the retrieved object store in the session context's runtime environment ctx.register_object_store(url, store); @@ -394,6 +395,7 @@ mod tests { use datafusion::common::config::FormatOptions; use datafusion::common::plan_err; + use datafusion::prelude::SessionContext; use url::Url; async fn create_external_table_test(location: &str, sql: &str) -> Result<()> { diff --git a/datafusion-cli/src/lib.rs b/datafusion-cli/src/lib.rs index 5081436aa6c5..fbfc9242a61d 100644 --- a/datafusion-cli/src/lib.rs +++ b/datafusion-cli/src/lib.rs @@ -19,6 +19,7 @@ pub const DATAFUSION_CLI_VERSION: &str = env!("CARGO_PKG_VERSION"); pub mod catalog; +pub mod cli_context; pub mod command; pub mod exec; pub mod functions; diff --git a/datafusion-cli/src/object_storage.rs b/datafusion-cli/src/object_storage.rs index 85e0009bd267..87eb04d113de 100644 --- a/datafusion-cli/src/object_storage.rs +++ b/datafusion-cli/src/object_storage.rs @@ -25,7 +25,6 @@ use datafusion::common::config::{ use datafusion::common::{config_err, exec_datafusion_err, exec_err}; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionState; -use datafusion::prelude::SessionContext; use async_trait::async_trait; use aws_credential_types::provider::ProvideCredentials; @@ -392,48 +391,6 @@ impl ConfigExtension for GcpOptions { const PREFIX: &'static str = "gcp"; } -/// Registers storage options for different cloud storage schemes in a given -/// session context. -/// -/// This function is responsible for extending the session context with specific -/// options based on the storage scheme being used. These options are essential -/// for handling interactions with different cloud storage services such as Amazon -/// S3, Alibaba Cloud OSS, Google Cloud Storage, etc. -/// -/// # Parameters -/// -/// * `ctx` - A mutable reference to the session context where table options are -/// to be registered. The session context holds configuration and environment -/// for the current session. -/// * `scheme` - A string slice that represents the cloud storage scheme. This -/// determines which set of options will be registered in the session context. -/// -/// # Supported Schemes -/// -/// * `s3` or `oss` - Registers `AwsOptions` which are configurations specific to -/// Amazon S3 and Alibaba Cloud OSS. -/// * `gs` or `gcs` - Registers `GcpOptions` which are configurations specific to -/// Google Cloud Storage. -/// -/// NOTE: This function will not perform any action when given an unsupported scheme. -pub(crate) fn register_options(ctx: &SessionContext, scheme: &str) { - // Match the provided scheme against supported cloud storage schemes: - match scheme { - // For Amazon S3 or Alibaba Cloud OSS - "s3" | "oss" | "cos" => { - // Register AWS specific table options in the session context: - ctx.register_table_options_extension(AwsOptions::default()) - } - // For Google Cloud Storage - "gs" | "gcs" => { - // Register GCP specific table options in the session context: - ctx.register_table_options_extension(GcpOptions::default()) - } - // For unsupported schemes, do nothing: - _ => {} - } -} - pub(crate) async fn get_object_store( state: &SessionState, scheme: &str, @@ -498,6 +455,8 @@ pub(crate) async fn get_object_store( #[cfg(test)] mod tests { + use crate::cli_context::CliSessionContext; + use super::*; use datafusion::common::plan_err; @@ -534,7 +493,7 @@ mod tests { let mut plan = ctx.state().create_logical_plan(&sql).await?; if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { - register_options(&ctx, scheme); + ctx.register_table_options_extension_from_scheme(scheme); let mut table_options = ctx.state().default_table_options().clone(); table_options.alter_with_string_hash_map(&cmd.options)?; let aws_options = table_options.extensions.get::().unwrap(); @@ -579,7 +538,7 @@ mod tests { let mut plan = ctx.state().create_logical_plan(&sql).await?; if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { - register_options(&ctx, scheme); + ctx.register_table_options_extension_from_scheme(scheme); let mut table_options = ctx.state().default_table_options().clone(); table_options.alter_with_string_hash_map(&cmd.options)?; let aws_options = table_options.extensions.get::().unwrap(); @@ -605,7 +564,7 @@ mod tests { let mut plan = ctx.state().create_logical_plan(&sql).await?; if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { - register_options(&ctx, scheme); + ctx.register_table_options_extension_from_scheme(scheme); let mut table_options = ctx.state().default_table_options().clone(); table_options.alter_with_string_hash_map(&cmd.options)?; let aws_options = table_options.extensions.get::().unwrap(); @@ -633,7 +592,7 @@ mod tests { let mut plan = ctx.state().create_logical_plan(&sql).await?; if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { - register_options(&ctx, scheme); + ctx.register_table_options_extension_from_scheme(scheme); let mut table_options = ctx.state().default_table_options().clone(); table_options.alter_with_string_hash_map(&cmd.options)?; let aws_options = table_options.extensions.get::().unwrap(); @@ -670,7 +629,7 @@ mod tests { let mut plan = ctx.state().create_logical_plan(&sql).await?; if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { - register_options(&ctx, scheme); + ctx.register_table_options_extension_from_scheme(scheme); let mut table_options = ctx.state().default_table_options().clone(); table_options.alter_with_string_hash_map(&cmd.options)?; let gcp_options = table_options.extensions.get::().unwrap(); From e1cfb48215ee91a183e06cfee602e42d2c23f429 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 17 Jun 2024 13:18:04 -0400 Subject: [PATCH 07/54] Minor: remove string copy from Column::from_qualified_name (#10947) --- datafusion/common/src/column.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index 911ff079def1..3e2bc0ad7c3a 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -109,21 +109,21 @@ impl Column { /// `foo.BAR` would be parsed to a reference to relation `foo`, column name `bar` (lower case) /// where `"foo.BAR"` would be parsed to a reference to column named `foo.BAR` pub fn from_qualified_name(flat_name: impl Into) -> Self { - let flat_name: &str = &flat_name.into(); - Self::from_idents(&mut parse_identifiers_normalized(flat_name, false)) + let flat_name = flat_name.into(); + Self::from_idents(&mut parse_identifiers_normalized(&flat_name, false)) .unwrap_or_else(|| Self { relation: None, - name: flat_name.to_owned(), + name: flat_name, }) } /// Deserialize a fully qualified name string into a column preserving column text case pub fn from_qualified_name_ignore_case(flat_name: impl Into) -> Self { - let flat_name: &str = &flat_name.into(); - Self::from_idents(&mut parse_identifiers_normalized(flat_name, true)) + let flat_name = flat_name.into(); + Self::from_idents(&mut parse_identifiers_normalized(&flat_name, true)) .unwrap_or_else(|| Self { relation: None, - name: flat_name.to_owned(), + name: flat_name, }) } From 1cb0057b99a016ee982d34deb860cf7f4e52b845 Mon Sep 17 00:00:00 2001 From: Marvin Lanhenke <62298609+marvinlanhenke@users.noreply.github.com> Date: Mon, 17 Jun 2024 21:22:48 +0200 Subject: [PATCH 08/54] Fix: StatisticsConverter `counts` for missing columns (#10946) * feat: add run_with_schema + add test_case * fix: null_counts * fix: row_counts * refactor: change return type of data_page_row_counts * refactor: shorten row_group_indices --- .../physical_plan/parquet/row_groups.rs | 1 + .../physical_plan/parquet/statistics.rs | 43 ++-- .../core/tests/parquet/arrow_statistics.rs | 231 +++++++++++------- 3 files changed, 167 insertions(+), 108 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 06eb8f79dada..f8e4889f0b7f 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -381,6 +381,7 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { self.statistics_converter(column) .and_then(|c| c.row_group_null_counts(self.metadata_iter())) .ok() + .map(|counts| Arc::new(counts) as ArrayRef) } fn row_counts(&self, _column: &Column) -> Option { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index a2f17ca9b7a7..14d7bc2af42d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -661,7 +661,7 @@ where /// of parquet page [`Index`]'es to an [`ArrayRef`] /// /// The returned Array is an [`UInt64Array`] -pub(crate) fn null_counts_page_statistics<'a, I>(iterator: I) -> Result +pub(crate) fn null_counts_page_statistics<'a, I>(iterator: I) -> Result where I: Iterator, { @@ -680,7 +680,7 @@ where _ => unimplemented!(), }); - Ok(Arc::new(UInt64Array::from_iter(iter))) + Ok(UInt64Array::from_iter(iter)) } /// Extracts Parquet statistics as Arrow arrays @@ -874,21 +874,22 @@ impl<'a> StatisticsConverter<'a> { /// Extract the null counts from row group statistics in [`RowGroupMetaData`] /// /// See docs on [`Self::row_group_mins`] for details - pub fn row_group_null_counts(&self, metadatas: I) -> Result + pub fn row_group_null_counts(&self, metadatas: I) -> Result where I: IntoIterator, { - let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, metadatas)); + let num_row_groups = metadatas.into_iter().count(); + return Ok(UInt64Array::from_iter( + std::iter::repeat(None).take(num_row_groups), + )); }; let null_counts = metadatas .into_iter() .map(|x| x.column(parquet_index).statistics()) .map(|s| s.map(|s| s.null_count())); - Ok(Arc::new(UInt64Array::from_iter(null_counts))) + Ok(UInt64Array::from_iter(null_counts)) } /// Extract the minimum values from Data Page statistics. @@ -1007,14 +1008,15 @@ impl<'a> StatisticsConverter<'a> { column_page_index: &ParquetColumnIndex, column_offset_index: &ParquetOffsetIndex, row_group_indices: I, - ) -> Result + ) -> Result where I: IntoIterator, { - let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, row_group_indices)); + let num_row_groups = row_group_indices.into_iter().count(); + return Ok(UInt64Array::from_iter( + std::iter::repeat(None).take(num_row_groups), + )); }; let iter = row_group_indices.into_iter().map(|rg_index| { @@ -1047,21 +1049,19 @@ impl<'a> StatisticsConverter<'a> { pub fn data_page_row_counts( &self, column_offset_index: &ParquetOffsetIndex, - row_group_metadatas: &[RowGroupMetaData], + row_group_metadatas: &'a [RowGroupMetaData], row_group_indices: I, - ) -> Result + ) -> Result> where I: IntoIterator, { - let data_type = self.arrow_field.data_type(); - let Some(parquet_index) = self.parquet_index else { - return Ok(self.make_null_array(data_type, row_group_indices)); + // no matching column found in parquet_index; + // thus we cannot extract page_locations in order to determine + // the row count on a per DataPage basis. + return Ok(None); }; - // `offset_index[row_group_number][column_number][page_number]` holds - // the [`PageLocation`] corresponding to page `page_number` of column - // `column_number`of row group `row_group_number`. let mut row_count_total = Vec::new(); for rg_idx in row_group_indices { let page_locations = &column_offset_index[*rg_idx][parquet_index]; @@ -1070,9 +1070,8 @@ impl<'a> StatisticsConverter<'a> { Some(loc[1].first_row_index as u64 - loc[0].first_row_index as u64) }); - let num_rows_in_row_group = &row_group_metadatas[*rg_idx].num_rows(); - // append the last page row count + let num_rows_in_row_group = &row_group_metadatas[*rg_idx].num_rows(); let row_count_per_page = row_count_per_page .chain(std::iter::once(Some( *num_rows_in_row_group as u64 @@ -1083,7 +1082,7 @@ impl<'a> StatisticsConverter<'a> { row_count_total.extend(row_count_per_page); } - Ok(Arc::new(UInt64Array::from_iter(row_count_total))) + Ok(Some(UInt64Array::from_iter(row_count_total))) } /// Returns a null array of data_type with one element per row group diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index 87bd1372225f..cd0efc8d3525 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -201,7 +201,7 @@ struct Test<'a> { expected_min: ArrayRef, expected_max: ArrayRef, expected_null_counts: UInt64Array, - expected_row_counts: UInt64Array, + expected_row_counts: Option, /// Which column to extract statistics from column_name: &'static str, /// What statistics should be checked? @@ -210,6 +210,28 @@ struct Test<'a> { impl<'a> Test<'a> { fn run(self) { + let converter = StatisticsConverter::try_new( + self.column_name, + self.reader.schema(), + self.reader.parquet_schema(), + ) + .unwrap(); + + self.run_checks(converter); + } + + fn run_with_schema(self, schema: &Schema) { + let converter = StatisticsConverter::try_new( + self.column_name, + schema, + self.reader.parquet_schema(), + ) + .unwrap(); + + self.run_checks(converter); + } + + fn run_checks(self, converter: StatisticsConverter) { let Self { reader, expected_min, @@ -220,15 +242,7 @@ impl<'a> Test<'a> { check, } = self; - let converter = StatisticsConverter::try_new( - column_name, - reader.schema(), - reader.parquet_schema(), - ) - .unwrap(); - let row_groups = reader.metadata().row_groups(); - let expected_null_counts = Arc::new(expected_null_counts) as ArrayRef; if check.data_page() { let column_page_index = reader @@ -241,11 +255,7 @@ impl<'a> Test<'a> { .offset_index() .expect("File should have column offset indices"); - let row_group_indices = row_groups - .iter() - .enumerate() - .map(|(i, _)| i) - .collect::>(); + let row_group_indices: Vec<_> = (0..row_groups.len()).collect(); let min = converter .data_page_mins( @@ -288,10 +298,8 @@ impl<'a> Test<'a> { let row_counts = converter .data_page_row_counts(column_offset_index, row_groups, &row_group_indices) .unwrap(); - // https://github.com/apache/datafusion/issues/10926 - let expected_row_counts: ArrayRef = Arc::new(expected_row_counts.clone()); assert_eq!( - &row_counts, &expected_row_counts, + row_counts, expected_row_counts, "{column_name}: Mismatch with expected row counts. \ Actual: {row_counts:?}. Expected: {expected_row_counts:?}" ); @@ -321,6 +329,7 @@ impl<'a> Test<'a> { reader.metadata().row_groups().iter(), ) .unwrap(); + let row_counts = Some(row_counts); assert_eq!( row_counts, expected_row_counts, "{column_name}: Mismatch with expected row counts. \ @@ -377,7 +386,7 @@ async fn test_one_row_group_without_null() { // no nulls expected_null_counts: UInt64Array::from(vec![0]), // 3 rows - expected_row_counts: UInt64Array::from(vec![3]), + expected_row_counts: Some(UInt64Array::from(vec![3])), column_name: "i64", check: Check::RowGroup, } @@ -404,7 +413,7 @@ async fn test_one_row_group_with_null_and_negative() { // 2 nulls expected_null_counts: UInt64Array::from(vec![2]), // 8 rows - expected_row_counts: UInt64Array::from(vec![8]), + expected_row_counts: Some(UInt64Array::from(vec![8])), column_name: "i64", check: Check::RowGroup, } @@ -431,7 +440,7 @@ async fn test_two_row_group_with_null() { // nulls are [0, 2] expected_null_counts: UInt64Array::from(vec![0, 2]), // row counts are [10, 5] - expected_row_counts: UInt64Array::from(vec![10, 5]), + expected_row_counts: Some(UInt64Array::from(vec![10, 5])), column_name: "i64", check: Check::RowGroup, } @@ -458,7 +467,7 @@ async fn test_two_row_groups_with_all_nulls_in_one() { // nulls are [1, 3] expected_null_counts: UInt64Array::from(vec![1, 3]), // row counts are [5, 3] - expected_row_counts: UInt64Array::from(vec![5, 3]), + expected_row_counts: Some(UInt64Array::from(vec![5, 3])), column_name: "i64", check: Check::RowGroup, } @@ -489,7 +498,7 @@ async fn test_multiple_data_pages_nulls_and_negatives() { expected_min: Arc::new(Int64Array::from(vec![Some(-1), Some(3), Some(7), None])), expected_max: Arc::new(Int64Array::from(vec![Some(2), Some(6), Some(9), None])), expected_null_counts: UInt64Array::from(vec![0, 0, 1, 2]), - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 2]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 2])), column_name: "i64", check: Check::DataPage, } @@ -522,7 +531,7 @@ async fn test_int_64() { // nulls are [0, 0, 0, 0] expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "i64", check: Check::Both, } @@ -548,7 +557,7 @@ async fn test_int_32() { // nulls are [0, 0, 0, 0] expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "i32", check: Check::Both, } @@ -574,7 +583,7 @@ async fn test_int_16() { // nulls are [0, 0, 0, 0] expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "i16", check: Check::Both, } @@ -600,7 +609,7 @@ async fn test_int_8() { // nulls are [0, 0, 0, 0] expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "i8", check: Check::Both, } @@ -650,7 +659,7 @@ async fn test_timestamp() { // nulls are [1, 1, 1, 1] expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "nanos", check: Check::RowGroup, } @@ -679,7 +688,7 @@ async fn test_timestamp() { // nulls are [1, 1, 1, 1] expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "nanos_timezoned", check: Check::RowGroup, } @@ -701,7 +710,7 @@ async fn test_timestamp() { TimestampMicrosecondType::parse("2020-01-12T01:01:01"), ])), expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "micros", check: Check::RowGroup, } @@ -730,7 +739,7 @@ async fn test_timestamp() { // nulls are [1, 1, 1, 1] expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "micros_timezoned", check: Check::RowGroup, } @@ -752,7 +761,7 @@ async fn test_timestamp() { TimestampMillisecondType::parse("2020-01-12T01:01:01"), ])), expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "millis", check: Check::RowGroup, } @@ -781,7 +790,7 @@ async fn test_timestamp() { // nulls are [1, 1, 1, 1] expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "millis_timezoned", check: Check::RowGroup, } @@ -803,7 +812,7 @@ async fn test_timestamp() { TimestampSecondType::parse("2020-01-12T01:01:01"), ])), expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "seconds", check: Check::RowGroup, } @@ -832,7 +841,7 @@ async fn test_timestamp() { // nulls are [1, 1, 1, 1] expected_null_counts: UInt64Array::from(vec![1, 1, 1, 1]), // row counts are [5, 5, 5, 5] - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "seconds_timezoned", check: Check::RowGroup, } @@ -878,7 +887,7 @@ async fn test_timestamp_diff_rg_sizes() { // nulls are [1, 2, 1] expected_null_counts: UInt64Array::from(vec![1, 2, 1]), // row counts are [8, 8, 4] - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "nanos", check: Check::RowGroup, } @@ -905,7 +914,7 @@ async fn test_timestamp_diff_rg_sizes() { // nulls are [1, 2, 1] expected_null_counts: UInt64Array::from(vec![1, 2, 1]), // row counts are [8, 8, 4] - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "nanos_timezoned", check: Check::RowGroup, } @@ -925,7 +934,7 @@ async fn test_timestamp_diff_rg_sizes() { TimestampMicrosecondType::parse("2020-01-12T01:01:01"), ])), expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "micros", check: Check::RowGroup, } @@ -952,7 +961,7 @@ async fn test_timestamp_diff_rg_sizes() { // nulls are [1, 2, 1] expected_null_counts: UInt64Array::from(vec![1, 2, 1]), // row counts are [8, 8, 4] - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "micros_timezoned", check: Check::RowGroup, } @@ -972,7 +981,7 @@ async fn test_timestamp_diff_rg_sizes() { TimestampMillisecondType::parse("2020-01-12T01:01:01"), ])), expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "millis", check: Check::RowGroup, } @@ -999,7 +1008,7 @@ async fn test_timestamp_diff_rg_sizes() { // nulls are [1, 2, 1] expected_null_counts: UInt64Array::from(vec![1, 2, 1]), // row counts are [8, 8, 4] - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "millis_timezoned", check: Check::RowGroup, } @@ -1019,7 +1028,7 @@ async fn test_timestamp_diff_rg_sizes() { TimestampSecondType::parse("2020-01-12T01:01:01"), ])), expected_null_counts: UInt64Array::from(vec![1, 2, 1]), - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "seconds", check: Check::RowGroup, } @@ -1046,7 +1055,7 @@ async fn test_timestamp_diff_rg_sizes() { // nulls are [1, 2, 1] expected_null_counts: UInt64Array::from(vec![1, 2, 1]), // row counts are [8, 8, 4] - expected_row_counts: UInt64Array::from(vec![8, 8, 4]), + expected_row_counts: Some(UInt64Array::from(vec![8, 8, 4])), column_name: "seconds_timezoned", check: Check::RowGroup, } @@ -1084,7 +1093,7 @@ async fn test_dates_32_diff_rg_sizes() { // nulls are [2, 2] expected_null_counts: UInt64Array::from(vec![2, 2]), // row counts are [13, 7] - expected_row_counts: UInt64Array::from(vec![13, 7]), + expected_row_counts: Some(UInt64Array::from(vec![13, 7])), column_name: "date32", check: Check::RowGroup, } @@ -1107,7 +1116,7 @@ async fn test_time32_second_diff_rg_sizes() { expected_min: Arc::new(Time32SecondArray::from(vec![18506, 18510, 18514, 18518])), expected_max: Arc::new(Time32SecondArray::from(vec![18509, 18513, 18517, 18521])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), column_name: "second", check: Check::RowGroup, } @@ -1134,7 +1143,7 @@ async fn test_time32_millisecond_diff_rg_sizes() { 3600003, 3600007, 3600011, 3600015, ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), column_name: "millisecond", check: Check::RowGroup, } @@ -1167,7 +1176,7 @@ async fn test_time64_microsecond_diff_rg_sizes() { 1234567890138, ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), column_name: "microsecond", check: Check::RowGroup, } @@ -1200,7 +1209,7 @@ async fn test_time64_nanosecond_diff_rg_sizes() { 987654321012360, ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), // Assuming 1 null per row group for simplicity - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4])), column_name: "nanosecond", check: Check::RowGroup, } @@ -1227,7 +1236,7 @@ async fn test_dates_64_diff_rg_sizes() { Date64Type::parse("2029-11-12"), ])), expected_null_counts: UInt64Array::from(vec![2, 2]), - expected_row_counts: UInt64Array::from(vec![13, 7]), + expected_row_counts: Some(UInt64Array::from(vec![13, 7])), column_name: "date64", check: Check::RowGroup, } @@ -1255,7 +1264,7 @@ async fn test_uint() { expected_min: Arc::new(UInt8Array::from(vec![0, 1, 4, 7, 251])), expected_max: Arc::new(UInt8Array::from(vec![3, 4, 6, 250, 254])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u8", check: Check::RowGroup, } @@ -1266,7 +1275,7 @@ async fn test_uint() { expected_min: Arc::new(UInt16Array::from(vec![0, 1, 4, 7, 251])), expected_max: Arc::new(UInt16Array::from(vec![3, 4, 6, 250, 254])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u16", check: Check::RowGroup, } @@ -1277,7 +1286,7 @@ async fn test_uint() { expected_min: Arc::new(UInt32Array::from(vec![0, 1, 4, 7, 251])), expected_max: Arc::new(UInt32Array::from(vec![3, 4, 6, 250, 254])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u32", check: Check::RowGroup, } @@ -1288,7 +1297,7 @@ async fn test_uint() { expected_min: Arc::new(UInt64Array::from(vec![0, 1, 4, 7, 251])), expected_max: Arc::new(UInt64Array::from(vec![3, 4, 6, 250, 254])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![4, 4, 4, 4, 4]), + expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u64", check: Check::RowGroup, } @@ -1311,7 +1320,7 @@ async fn test_int32_range() { expected_min: Arc::new(Int32Array::from(vec![0])), expected_max: Arc::new(Int32Array::from(vec![300000])), expected_null_counts: UInt64Array::from(vec![0]), - expected_row_counts: UInt64Array::from(vec![4]), + expected_row_counts: Some(UInt64Array::from(vec![4])), column_name: "i", check: Check::RowGroup, } @@ -1334,7 +1343,7 @@ async fn test_uint32_range() { expected_min: Arc::new(UInt32Array::from(vec![0])), expected_max: Arc::new(UInt32Array::from(vec![300000])), expected_null_counts: UInt64Array::from(vec![0]), - expected_row_counts: UInt64Array::from(vec![4]), + expected_row_counts: Some(UInt64Array::from(vec![4])), column_name: "u", check: Check::RowGroup, } @@ -1356,7 +1365,7 @@ async fn test_numeric_limits_unsigned() { expected_min: Arc::new(UInt8Array::from(vec![u8::MIN, 100])), expected_max: Arc::new(UInt8Array::from(vec![100, u8::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "u8", check: Check::RowGroup, } @@ -1367,7 +1376,7 @@ async fn test_numeric_limits_unsigned() { expected_min: Arc::new(UInt16Array::from(vec![u16::MIN, 100])), expected_max: Arc::new(UInt16Array::from(vec![100, u16::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "u16", check: Check::RowGroup, } @@ -1378,7 +1387,7 @@ async fn test_numeric_limits_unsigned() { expected_min: Arc::new(UInt32Array::from(vec![u32::MIN, 100])), expected_max: Arc::new(UInt32Array::from(vec![100, u32::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "u32", check: Check::RowGroup, } @@ -1389,7 +1398,7 @@ async fn test_numeric_limits_unsigned() { expected_min: Arc::new(UInt64Array::from(vec![u64::MIN, 100])), expected_max: Arc::new(UInt64Array::from(vec![100, u64::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "u64", check: Check::RowGroup, } @@ -1411,7 +1420,7 @@ async fn test_numeric_limits_signed() { expected_min: Arc::new(Int8Array::from(vec![i8::MIN, -100])), expected_max: Arc::new(Int8Array::from(vec![100, i8::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "i8", check: Check::RowGroup, } @@ -1422,7 +1431,7 @@ async fn test_numeric_limits_signed() { expected_min: Arc::new(Int16Array::from(vec![i16::MIN, -100])), expected_max: Arc::new(Int16Array::from(vec![100, i16::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "i16", check: Check::RowGroup, } @@ -1433,7 +1442,7 @@ async fn test_numeric_limits_signed() { expected_min: Arc::new(Int32Array::from(vec![i32::MIN, -100])), expected_max: Arc::new(Int32Array::from(vec![100, i32::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "i32", check: Check::RowGroup, } @@ -1444,7 +1453,7 @@ async fn test_numeric_limits_signed() { expected_min: Arc::new(Int64Array::from(vec![i64::MIN, -100])), expected_max: Arc::new(Int64Array::from(vec![100, i64::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "i64", check: Check::RowGroup, } @@ -1466,7 +1475,7 @@ async fn test_numeric_limits_float() { expected_min: Arc::new(Float32Array::from(vec![f32::MIN, -100.0])), expected_max: Arc::new(Float32Array::from(vec![100.0, f32::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "f32", check: Check::RowGroup, } @@ -1477,7 +1486,7 @@ async fn test_numeric_limits_float() { expected_min: Arc::new(Float64Array::from(vec![f64::MIN, -100.0])), expected_max: Arc::new(Float64Array::from(vec![100.0, f64::MAX])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "f64", check: Check::RowGroup, } @@ -1488,7 +1497,7 @@ async fn test_numeric_limits_float() { expected_min: Arc::new(Float32Array::from(vec![-1.0, -100.0])), expected_max: Arc::new(Float32Array::from(vec![100.0, -100.0])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "f32_nan", check: Check::RowGroup, } @@ -1499,7 +1508,7 @@ async fn test_numeric_limits_float() { expected_min: Arc::new(Float64Array::from(vec![-1.0, -100.0])), expected_max: Arc::new(Float64Array::from(vec![100.0, -100.0])), expected_null_counts: UInt64Array::from(vec![0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "f64_nan", check: Check::RowGroup, } @@ -1522,7 +1531,7 @@ async fn test_float64() { expected_min: Arc::new(Float64Array::from(vec![-5.0, -4.0, -0.0, 5.0])), expected_max: Arc::new(Float64Array::from(vec![-1.0, 0.0, 4.0, 9.0])), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "f", check: Check::RowGroup, } @@ -1555,7 +1564,7 @@ async fn test_float16() { .collect::>(), )), expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), column_name: "f", check: Check::RowGroup, } @@ -1586,7 +1595,7 @@ async fn test_decimal() { .unwrap(), ), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "decimal_col", check: Check::RowGroup, } @@ -1624,7 +1633,7 @@ async fn test_decimal_256() { .unwrap(), ), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "decimal256_col", check: Check::RowGroup, } @@ -1644,7 +1653,7 @@ async fn test_dictionary() { expected_min: Arc::new(StringArray::from(vec!["abc", "aaa"])), expected_max: Arc::new(StringArray::from(vec!["def", "fffff"])), expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "string_dict_i8", check: Check::RowGroup, } @@ -1655,7 +1664,7 @@ async fn test_dictionary() { expected_min: Arc::new(StringArray::from(vec!["abc", "aaa"])), expected_max: Arc::new(StringArray::from(vec!["def", "fffff"])), expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "string_dict_i32", check: Check::RowGroup, } @@ -1666,7 +1675,7 @@ async fn test_dictionary() { expected_min: Arc::new(Int64Array::from(vec![-100, 0])), expected_max: Arc::new(Int64Array::from(vec![0, 100])), expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: UInt64Array::from(vec![5, 2]), + expected_row_counts: Some(UInt64Array::from(vec![5, 2])), column_name: "int_dict_i8", check: Check::RowGroup, } @@ -1704,7 +1713,7 @@ async fn test_byte() { "all backends", ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "name", check: Check::RowGroup, } @@ -1724,7 +1733,7 @@ async fn test_byte() { "backend six", ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "service_string", check: Check::RowGroup, } @@ -1743,7 +1752,7 @@ async fn test_byte() { expected_min: Arc::new(BinaryArray::from(expected_service_binary_min_values)), expected_max: Arc::new(BinaryArray::from(expected_service_binary_max_values)), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "service_binary", check: Check::RowGroup, } @@ -1764,7 +1773,7 @@ async fn test_byte() { FixedSizeBinaryArray::try_from_iter(max_input.into_iter()).unwrap(), ), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "service_fixedsize", check: Check::RowGroup, } @@ -1785,7 +1794,7 @@ async fn test_byte() { expected_service_large_binary_max_values, )), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "service_large_binary", check: Check::RowGroup, } @@ -1818,7 +1827,7 @@ async fn test_period_in_column_names() { "HTTP GET / DISPATCH", ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "name", check: Check::RowGroup, } @@ -1832,7 +1841,7 @@ async fn test_period_in_column_names() { "frontend", "frontend", "backend", ])), expected_null_counts: UInt64Array::from(vec![0, 0, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5])), column_name: "service.name", check: Check::RowGroup, } @@ -1856,7 +1865,7 @@ async fn test_boolean() { expected_min: Arc::new(BooleanArray::from(vec![false, false])), expected_max: Arc::new(BooleanArray::from(vec![true, false])), expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5])), column_name: "bool", check: Check::RowGroup, } @@ -1883,7 +1892,7 @@ async fn test_struct() { expected_min: Arc::new(struct_array(vec![(Some(1), Some(6.0), Some(12.0))])), expected_max: Arc::new(struct_array(vec![(Some(2), Some(8.5), Some(14.0))])), expected_null_counts: UInt64Array::from(vec![0]), - expected_row_counts: UInt64Array::from(vec![3]), + expected_row_counts: Some(UInt64Array::from(vec![3])), column_name: "struct", check: Check::RowGroup, } @@ -1906,7 +1915,7 @@ async fn test_utf8() { expected_min: Arc::new(StringArray::from(vec!["a", "e"])), expected_max: Arc::new(StringArray::from(vec!["d", "i"])), expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5])), column_name: "utf8", check: Check::RowGroup, } @@ -1918,7 +1927,7 @@ async fn test_utf8() { expected_min: Arc::new(LargeStringArray::from(vec!["a", "e"])), expected_max: Arc::new(LargeStringArray::from(vec!["d", "i"])), expected_null_counts: UInt64Array::from(vec![1, 0]), - expected_row_counts: UInt64Array::from(vec![5, 5]), + expected_row_counts: Some(UInt64Array::from(vec![5, 5])), column_name: "large_utf8", check: Check::RowGroup, } @@ -1944,7 +1953,7 @@ async fn test_missing_statistics() { expected_min: Arc::new(Int64Array::from(vec![None])), expected_max: Arc::new(Int64Array::from(vec![None])), expected_null_counts: UInt64Array::from(vec![None]), - expected_row_counts: UInt64Array::from(vec![3]), // stil has row count statistics + expected_row_counts: Some(UInt64Array::from(vec![3])), // stil has row count statistics column_name: "i64", check: Check::RowGroup, } @@ -1966,9 +1975,59 @@ async fn test_column_not_found() { expected_min: Arc::new(Int64Array::from(vec![18262, 18565])), expected_max: Arc::new(Int64Array::from(vec![18564, 21865])), expected_null_counts: UInt64Array::from(vec![2, 2]), - expected_row_counts: UInt64Array::from(vec![13, 7]), + expected_row_counts: Some(UInt64Array::from(vec![13, 7])), column_name: "not_a_column", check: Check::RowGroup, } .run_col_not_found(); } + +#[tokio::test] +async fn test_column_non_existent() { + // Create a schema with an additional column + // that will not have a matching parquet index + let schema = Arc::new(Schema::new(vec![ + Field::new("i8", DataType::Int8, true), + Field::new("i16", DataType::Int16, true), + Field::new("i32", DataType::Int32, true), + Field::new("i64", DataType::Int64, true), + Field::new("i_do_not_exist", DataType::Int64, true), + ])); + + let reader = TestReader { + scenario: Scenario::Int, + row_per_group: 5, + } + .build() + .await; + + Test { + reader: &reader, + // mins are [-5, -4, 0, 5] + expected_min: Arc::new(Int64Array::from(vec![None, None, None, None])), + // maxes are [-1, 0, 4, 9] + expected_max: Arc::new(Int64Array::from(vec![None, None, None, None])), + // nulls are [0, 0, 0, 0] + expected_null_counts: UInt64Array::from(vec![None, None, None, None]), + // row counts are [5, 5, 5, 5] + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), + column_name: "i_do_not_exist", + check: Check::RowGroup, + } + .run_with_schema(&schema); + + Test { + reader: &reader, + // mins are [-5, -4, 0, 5] + expected_min: Arc::new(Int64Array::from(vec![None, None, None, None])), + // maxes are [-1, 0, 4, 9] + expected_max: Arc::new(Int64Array::from(vec![None, None, None, None])), + // nulls are [0, 0, 0, 0] + expected_null_counts: UInt64Array::from(vec![None, None, None, None]), + // row counts are [5, 5, 5, 5] + expected_row_counts: None, + column_name: "i_do_not_exist", + check: Check::DataPage, + } + .run_with_schema(&schema); +} From f373a866ce849679f6726699e795ffcb6638609c Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Mon, 17 Jun 2024 15:31:21 -0400 Subject: [PATCH 09/54] Add initial support for Utf8View and BinaryView types (#10925) * add view types * Add slt tests * comment out failing test * update vendored code --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/scalar/mod.rs | 93 ++++++++++++++++--- datafusion/functions/src/core/arrow_cast.rs | 4 + .../proto/datafusion_common.proto | 4 + datafusion/proto-common/src/from_proto/mod.rs | 4 + .../proto-common/src/generated/pbjson.rs | 55 +++++++++++ .../proto-common/src/generated/prost.rs | 12 ++- datafusion/proto-common/src/to_proto/mod.rs | 10 ++ .../src/generated/datafusion_proto_common.rs | 12 ++- datafusion/sql/src/unparser/expr.rs | 8 ++ .../sqllogictest/test_files/arrow_typeof.slt | 10 ++ 10 files changed, 193 insertions(+), 19 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 8073b21cdde0..3daf347ae4ff 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -221,10 +221,14 @@ pub enum ScalarValue { UInt64(Option), /// utf-8 encoded string. Utf8(Option), + /// utf-8 encoded string but from view types. + Utf8View(Option), /// utf-8 encoded string representing a LargeString's arrow type. LargeUtf8(Option), /// binary Binary(Option>), + /// binary but from view types. + BinaryView(Option>), /// fixed size binary FixedSizeBinary(i32, Option>), /// large binary @@ -345,10 +349,14 @@ impl PartialEq for ScalarValue { (UInt64(_), _) => false, (Utf8(v1), Utf8(v2)) => v1.eq(v2), (Utf8(_), _) => false, + (Utf8View(v1), Utf8View(v2)) => v1.eq(v2), + (Utf8View(_), _) => false, (LargeUtf8(v1), LargeUtf8(v2)) => v1.eq(v2), (LargeUtf8(_), _) => false, (Binary(v1), Binary(v2)) => v1.eq(v2), (Binary(_), _) => false, + (BinaryView(v1), BinaryView(v2)) => v1.eq(v2), + (BinaryView(_), _) => false, (FixedSizeBinary(_, v1), FixedSizeBinary(_, v2)) => v1.eq(v2), (FixedSizeBinary(_, _), _) => false, (LargeBinary(v1), LargeBinary(v2)) => v1.eq(v2), @@ -470,8 +478,12 @@ impl PartialOrd for ScalarValue { (Utf8(_), _) => None, (LargeUtf8(v1), LargeUtf8(v2)) => v1.partial_cmp(v2), (LargeUtf8(_), _) => None, + (Utf8View(v1), Utf8View(v2)) => v1.partial_cmp(v2), + (Utf8View(_), _) => None, (Binary(v1), Binary(v2)) => v1.partial_cmp(v2), (Binary(_), _) => None, + (BinaryView(v1), BinaryView(v2)) => v1.partial_cmp(v2), + (BinaryView(_), _) => None, (FixedSizeBinary(_, v1), FixedSizeBinary(_, v2)) => v1.partial_cmp(v2), (FixedSizeBinary(_, _), _) => None, (LargeBinary(v1), LargeBinary(v2)) => v1.partial_cmp(v2), @@ -667,11 +679,10 @@ impl std::hash::Hash for ScalarValue { UInt16(v) => v.hash(state), UInt32(v) => v.hash(state), UInt64(v) => v.hash(state), - Utf8(v) => v.hash(state), - LargeUtf8(v) => v.hash(state), - Binary(v) => v.hash(state), - FixedSizeBinary(_, v) => v.hash(state), - LargeBinary(v) => v.hash(state), + Utf8(v) | LargeUtf8(v) | Utf8View(v) => v.hash(state), + Binary(v) | FixedSizeBinary(_, v) | LargeBinary(v) | BinaryView(v) => { + v.hash(state) + } List(arr) => { hash_nested_array(arr.to_owned() as ArrayRef, state); } @@ -1107,7 +1118,9 @@ impl ScalarValue { ScalarValue::Float64(_) => DataType::Float64, ScalarValue::Utf8(_) => DataType::Utf8, ScalarValue::LargeUtf8(_) => DataType::LargeUtf8, + ScalarValue::Utf8View(_) => DataType::Utf8View, ScalarValue::Binary(_) => DataType::Binary, + ScalarValue::BinaryView(_) => DataType::BinaryView, ScalarValue::FixedSizeBinary(sz, _) => DataType::FixedSizeBinary(*sz), ScalarValue::LargeBinary(_) => DataType::LargeBinary, ScalarValue::List(arr) => arr.data_type().to_owned(), @@ -1310,11 +1323,13 @@ impl ScalarValue { ScalarValue::UInt16(v) => v.is_none(), ScalarValue::UInt32(v) => v.is_none(), ScalarValue::UInt64(v) => v.is_none(), - ScalarValue::Utf8(v) => v.is_none(), - ScalarValue::LargeUtf8(v) => v.is_none(), - ScalarValue::Binary(v) => v.is_none(), - ScalarValue::FixedSizeBinary(_, v) => v.is_none(), - ScalarValue::LargeBinary(v) => v.is_none(), + ScalarValue::Utf8(v) + | ScalarValue::Utf8View(v) + | ScalarValue::LargeUtf8(v) => v.is_none(), + ScalarValue::Binary(v) + | ScalarValue::BinaryView(v) + | ScalarValue::FixedSizeBinary(_, v) + | ScalarValue::LargeBinary(v) => v.is_none(), // arr.len() should be 1 for a list scalar, but we don't seem to // enforce that anywhere, so we still check against array length. ScalarValue::List(arr) => arr.len() == arr.null_count(), @@ -2002,6 +2017,12 @@ impl ScalarValue { } None => new_null_array(&DataType::Utf8, size), }, + ScalarValue::Utf8View(e) => match e { + Some(value) => { + Arc::new(StringViewArray::from_iter_values(repeat(value).take(size))) + } + None => new_null_array(&DataType::Utf8View, size), + }, ScalarValue::LargeUtf8(e) => match e { Some(value) => { Arc::new(LargeStringArray::from_iter_values(repeat(value).take(size))) @@ -2018,6 +2039,16 @@ impl ScalarValue { Arc::new(repeat(None::<&str>).take(size).collect::()) } }, + ScalarValue::BinaryView(e) => match e { + Some(value) => Arc::new( + repeat(Some(value.as_slice())) + .take(size) + .collect::(), + ), + None => { + Arc::new(repeat(None::<&str>).take(size).collect::()) + } + }, ScalarValue::FixedSizeBinary(s, e) => match e { Some(value) => Arc::new( FixedSizeBinaryArray::try_from_sparse_iter_with_size( @@ -2361,10 +2392,14 @@ impl ScalarValue { DataType::LargeBinary => { typed_cast!(array, index, LargeBinaryArray, LargeBinary)? } + DataType::BinaryView => { + typed_cast!(array, index, BinaryViewArray, BinaryView)? + } DataType::Utf8 => typed_cast!(array, index, StringArray, Utf8)?, DataType::LargeUtf8 => { typed_cast!(array, index, LargeStringArray, LargeUtf8)? } + DataType::Utf8View => typed_cast!(array, index, StringViewArray, Utf8View)?, DataType::List(_) => { let list_array = array.as_list::(); let nested_array = list_array.value(index); @@ -2652,12 +2687,18 @@ impl ScalarValue { ScalarValue::Utf8(val) => { eq_array_primitive!(array, index, StringArray, val)? } + ScalarValue::Utf8View(val) => { + eq_array_primitive!(array, index, StringViewArray, val)? + } ScalarValue::LargeUtf8(val) => { eq_array_primitive!(array, index, LargeStringArray, val)? } ScalarValue::Binary(val) => { eq_array_primitive!(array, index, BinaryArray, val)? } + ScalarValue::BinaryView(val) => { + eq_array_primitive!(array, index, BinaryViewArray, val)? + } ScalarValue::FixedSizeBinary(_, val) => { eq_array_primitive!(array, index, FixedSizeBinaryArray, val)? } @@ -2790,7 +2831,9 @@ impl ScalarValue { | ScalarValue::DurationMillisecond(_) | ScalarValue::DurationMicrosecond(_) | ScalarValue::DurationNanosecond(_) => 0, - ScalarValue::Utf8(s) | ScalarValue::LargeUtf8(s) => { + ScalarValue::Utf8(s) + | ScalarValue::LargeUtf8(s) + | ScalarValue::Utf8View(s) => { s.as_ref().map(|s| s.capacity()).unwrap_or_default() } ScalarValue::TimestampSecond(_, s) @@ -2801,7 +2844,8 @@ impl ScalarValue { } ScalarValue::Binary(b) | ScalarValue::FixedSizeBinary(_, b) - | ScalarValue::LargeBinary(b) => { + | ScalarValue::LargeBinary(b) + | ScalarValue::BinaryView(b) => { b.as_ref().map(|b| b.capacity()).unwrap_or_default() } ScalarValue::List(arr) => arr.get_array_memory_size(), @@ -3068,7 +3112,9 @@ impl TryFrom<&DataType> for ScalarValue { } DataType::Utf8 => ScalarValue::Utf8(None), DataType::LargeUtf8 => ScalarValue::LargeUtf8(None), + DataType::Utf8View => ScalarValue::Utf8View(None), DataType::Binary => ScalarValue::Binary(None), + DataType::BinaryView => ScalarValue::BinaryView(None), DataType::FixedSizeBinary(len) => ScalarValue::FixedSizeBinary(*len, None), DataType::LargeBinary => ScalarValue::LargeBinary(None), DataType::Date32 => ScalarValue::Date32(None), @@ -3190,11 +3236,13 @@ impl fmt::Display for ScalarValue { ScalarValue::TimestampMillisecond(e, _) => format_option!(f, e)?, ScalarValue::TimestampMicrosecond(e, _) => format_option!(f, e)?, ScalarValue::TimestampNanosecond(e, _) => format_option!(f, e)?, - ScalarValue::Utf8(e) => format_option!(f, e)?, - ScalarValue::LargeUtf8(e) => format_option!(f, e)?, + ScalarValue::Utf8(e) + | ScalarValue::LargeUtf8(e) + | ScalarValue::Utf8View(e) => format_option!(f, e)?, ScalarValue::Binary(e) | ScalarValue::FixedSizeBinary(_, e) - | ScalarValue::LargeBinary(e) => match e { + | ScalarValue::LargeBinary(e) + | ScalarValue::BinaryView(e) => match e { Some(l) => write!( f, "{}", @@ -3318,10 +3366,14 @@ impl fmt::Debug for ScalarValue { } ScalarValue::Utf8(None) => write!(f, "Utf8({self})"), ScalarValue::Utf8(Some(_)) => write!(f, "Utf8(\"{self}\")"), + ScalarValue::Utf8View(None) => write!(f, "Utf8View({self})"), + ScalarValue::Utf8View(Some(_)) => write!(f, "Utf8View(\"{self}\")"), ScalarValue::LargeUtf8(None) => write!(f, "LargeUtf8({self})"), ScalarValue::LargeUtf8(Some(_)) => write!(f, "LargeUtf8(\"{self}\")"), ScalarValue::Binary(None) => write!(f, "Binary({self})"), ScalarValue::Binary(Some(_)) => write!(f, "Binary(\"{self}\")"), + ScalarValue::BinaryView(None) => write!(f, "BinaryView({self})"), + ScalarValue::BinaryView(Some(_)) => write!(f, "BinaryView(\"{self}\")"), ScalarValue::FixedSizeBinary(size, None) => { write!(f, "FixedSizeBinary({size}, {self})") } @@ -5393,6 +5445,17 @@ mod tests { ScalarValue::Utf8(None), DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), ); + + // needs https://github.com/apache/arrow-rs/issues/5893 + /* + check_scalar_cast(ScalarValue::Utf8(None), DataType::Utf8View); + check_scalar_cast(ScalarValue::from("foo"), DataType::Utf8View); + check_scalar_cast( + ScalarValue::from("larger than 12 bytes string"), + DataType::Utf8View, + ); + + */ } // mimics how casting work on scalar values by `casting` `scalar` to `desired_type` diff --git a/datafusion/functions/src/core/arrow_cast.rs b/datafusion/functions/src/core/arrow_cast.rs index d641389e0ae3..9c410d4e18e8 100644 --- a/datafusion/functions/src/core/arrow_cast.rs +++ b/datafusion/functions/src/core/arrow_cast.rs @@ -564,7 +564,9 @@ impl<'a> Tokenizer<'a> { "Utf8" => Token::SimpleType(DataType::Utf8), "LargeUtf8" => Token::SimpleType(DataType::LargeUtf8), + "Utf8View" => Token::SimpleType(DataType::Utf8View), "Binary" => Token::SimpleType(DataType::Binary), + "BinaryView" => Token::SimpleType(DataType::BinaryView), "LargeBinary" => Token::SimpleType(DataType::LargeBinary), "Float16" => Token::SimpleType(DataType::Float16), @@ -772,11 +774,13 @@ mod test { DataType::Interval(IntervalUnit::DayTime), DataType::Interval(IntervalUnit::MonthDayNano), DataType::Binary, + DataType::BinaryView, DataType::FixedSizeBinary(0), DataType::FixedSizeBinary(1234), DataType::FixedSizeBinary(-432), DataType::LargeBinary, DataType::Utf8, + DataType::Utf8View, DataType::LargeUtf8, DataType::Decimal128(7, 12), DataType::Decimal256(6, 13), diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 29a348283f46..e523ef1a5e93 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -248,6 +248,7 @@ message ScalarValue{ bool bool_value = 1; string utf8_value = 2; string large_utf8_value = 3; + string utf8_view_value = 23; int32 int8_value = 4; int32 int16_value = 5; int32 int32_value = 6; @@ -281,6 +282,7 @@ message ScalarValue{ ScalarDictionaryValue dictionary_value = 27; bytes binary_value = 28; bytes large_binary_value = 29; + bytes binary_view_value = 22; ScalarTime64Value time64_value = 30; IntervalDayTimeValue interval_daytime_value = 25; IntervalMonthDayNanoValue interval_month_day_nano = 31; @@ -318,8 +320,10 @@ message ArrowType{ EmptyMessage FLOAT32 = 12 ; EmptyMessage FLOAT64 = 13 ; EmptyMessage UTF8 = 14 ; + EmptyMessage UTF8_VIEW = 35; EmptyMessage LARGE_UTF8 = 32; EmptyMessage BINARY = 15 ; + EmptyMessage BINARY_VIEW = 34; int32 FIXED_SIZE_BINARY = 16 ; EmptyMessage LARGE_BINARY = 31; EmptyMessage DATE32 = 17 ; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 25c1502ee75b..be87123fb13f 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -224,8 +224,10 @@ impl TryFrom<&protobuf::arrow_type::ArrowTypeEnum> for DataType { arrow_type::ArrowTypeEnum::Float32(_) => DataType::Float32, arrow_type::ArrowTypeEnum::Float64(_) => DataType::Float64, arrow_type::ArrowTypeEnum::Utf8(_) => DataType::Utf8, + arrow_type::ArrowTypeEnum::Utf8View(_) => DataType::Utf8View, arrow_type::ArrowTypeEnum::LargeUtf8(_) => DataType::LargeUtf8, arrow_type::ArrowTypeEnum::Binary(_) => DataType::Binary, + arrow_type::ArrowTypeEnum::BinaryView(_) => DataType::BinaryView, arrow_type::ArrowTypeEnum::FixedSizeBinary(size) => { DataType::FixedSizeBinary(*size) } @@ -361,6 +363,7 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { Ok(match value { Value::BoolValue(v) => Self::Boolean(Some(*v)), Value::Utf8Value(v) => Self::Utf8(Some(v.to_owned())), + Value::Utf8ViewValue(v) => Self::Utf8View(Some(v.to_owned())), Value::LargeUtf8Value(v) => Self::LargeUtf8(Some(v.to_owned())), Value::Int8Value(v) => Self::Int8(Some(*v as i8)), Value::Int16Value(v) => Self::Int16(Some(*v as i16)), @@ -571,6 +574,7 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { Self::Dictionary(Box::new(index_type), Box::new(value)) } Value::BinaryValue(v) => Self::Binary(Some(v.clone())), + Value::BinaryViewValue(v) => Self::BinaryView(Some(v.clone())), Value::LargeBinaryValue(v) => Self::LargeBinary(Some(v.clone())), Value::IntervalDaytimeValue(v) => Self::IntervalDayTime(Some( IntervalDayTimeType::make_value(v.days, v.milliseconds), diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 6f8409b82afe..ead29d9b92e0 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -125,12 +125,18 @@ impl serde::Serialize for ArrowType { arrow_type::ArrowTypeEnum::Utf8(v) => { struct_ser.serialize_field("UTF8", v)?; } + arrow_type::ArrowTypeEnum::Utf8View(v) => { + struct_ser.serialize_field("UTF8VIEW", v)?; + } arrow_type::ArrowTypeEnum::LargeUtf8(v) => { struct_ser.serialize_field("LARGEUTF8", v)?; } arrow_type::ArrowTypeEnum::Binary(v) => { struct_ser.serialize_field("BINARY", v)?; } + arrow_type::ArrowTypeEnum::BinaryView(v) => { + struct_ser.serialize_field("BINARYVIEW", v)?; + } arrow_type::ArrowTypeEnum::FixedSizeBinary(v) => { struct_ser.serialize_field("FIXEDSIZEBINARY", v)?; } @@ -216,9 +222,13 @@ impl<'de> serde::Deserialize<'de> for ArrowType { "FLOAT32", "FLOAT64", "UTF8", + "UTF8_VIEW", + "UTF8VIEW", "LARGE_UTF8", "LARGEUTF8", "BINARY", + "BINARY_VIEW", + "BINARYVIEW", "FIXED_SIZE_BINARY", "FIXEDSIZEBINARY", "LARGE_BINARY", @@ -258,8 +268,10 @@ impl<'de> serde::Deserialize<'de> for ArrowType { Float32, Float64, Utf8, + Utf8View, LargeUtf8, Binary, + BinaryView, FixedSizeBinary, LargeBinary, Date32, @@ -312,8 +324,10 @@ impl<'de> serde::Deserialize<'de> for ArrowType { "FLOAT32" => Ok(GeneratedField::Float32), "FLOAT64" => Ok(GeneratedField::Float64), "UTF8" => Ok(GeneratedField::Utf8), + "UTF8VIEW" | "UTF8_VIEW" => Ok(GeneratedField::Utf8View), "LARGEUTF8" | "LARGE_UTF8" => Ok(GeneratedField::LargeUtf8), "BINARY" => Ok(GeneratedField::Binary), + "BINARYVIEW" | "BINARY_VIEW" => Ok(GeneratedField::BinaryView), "FIXEDSIZEBINARY" | "FIXED_SIZE_BINARY" => Ok(GeneratedField::FixedSizeBinary), "LARGEBINARY" | "LARGE_BINARY" => Ok(GeneratedField::LargeBinary), "DATE32" => Ok(GeneratedField::Date32), @@ -449,6 +463,13 @@ impl<'de> serde::Deserialize<'de> for ArrowType { return Err(serde::de::Error::duplicate_field("UTF8")); } arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Utf8) +; + } + GeneratedField::Utf8View => { + if arrow_type_enum__.is_some() { + return Err(serde::de::Error::duplicate_field("UTF8VIEW")); + } + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Utf8View) ; } GeneratedField::LargeUtf8 => { @@ -463,6 +484,13 @@ impl<'de> serde::Deserialize<'de> for ArrowType { return Err(serde::de::Error::duplicate_field("BINARY")); } arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Binary) +; + } + GeneratedField::BinaryView => { + if arrow_type_enum__.is_some() { + return Err(serde::de::Error::duplicate_field("BINARYVIEW")); + } + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::BinaryView) ; } GeneratedField::FixedSizeBinary => { @@ -6255,6 +6283,9 @@ impl serde::Serialize for ScalarValue { scalar_value::Value::LargeUtf8Value(v) => { struct_ser.serialize_field("largeUtf8Value", v)?; } + scalar_value::Value::Utf8ViewValue(v) => { + struct_ser.serialize_field("utf8ViewValue", v)?; + } scalar_value::Value::Int8Value(v) => { struct_ser.serialize_field("int8Value", v)?; } @@ -6348,6 +6379,10 @@ impl serde::Serialize for ScalarValue { #[allow(clippy::needless_borrow)] struct_ser.serialize_field("largeBinaryValue", pbjson::private::base64::encode(&v).as_str())?; } + scalar_value::Value::BinaryViewValue(v) => { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("binaryViewValue", pbjson::private::base64::encode(&v).as_str())?; + } scalar_value::Value::Time64Value(v) => { struct_ser.serialize_field("time64Value", v)?; } @@ -6383,6 +6418,8 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "utf8Value", "large_utf8_value", "largeUtf8Value", + "utf8_view_value", + "utf8ViewValue", "int8_value", "int8Value", "int16_value", @@ -6439,6 +6476,8 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "binaryValue", "large_binary_value", "largeBinaryValue", + "binary_view_value", + "binaryViewValue", "time64_value", "time64Value", "interval_daytime_value", @@ -6457,6 +6496,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { BoolValue, Utf8Value, LargeUtf8Value, + Utf8ViewValue, Int8Value, Int16Value, Int32Value, @@ -6485,6 +6525,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { DictionaryValue, BinaryValue, LargeBinaryValue, + BinaryViewValue, Time64Value, IntervalDaytimeValue, IntervalMonthDayNano, @@ -6515,6 +6556,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "boolValue" | "bool_value" => Ok(GeneratedField::BoolValue), "utf8Value" | "utf8_value" => Ok(GeneratedField::Utf8Value), "largeUtf8Value" | "large_utf8_value" => Ok(GeneratedField::LargeUtf8Value), + "utf8ViewValue" | "utf8_view_value" => Ok(GeneratedField::Utf8ViewValue), "int8Value" | "int8_value" => Ok(GeneratedField::Int8Value), "int16Value" | "int16_value" => Ok(GeneratedField::Int16Value), "int32Value" | "int32_value" => Ok(GeneratedField::Int32Value), @@ -6543,6 +6585,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "dictionaryValue" | "dictionary_value" => Ok(GeneratedField::DictionaryValue), "binaryValue" | "binary_value" => Ok(GeneratedField::BinaryValue), "largeBinaryValue" | "large_binary_value" => Ok(GeneratedField::LargeBinaryValue), + "binaryViewValue" | "binary_view_value" => Ok(GeneratedField::BinaryViewValue), "time64Value" | "time64_value" => Ok(GeneratedField::Time64Value), "intervalDaytimeValue" | "interval_daytime_value" => Ok(GeneratedField::IntervalDaytimeValue), "intervalMonthDayNano" | "interval_month_day_nano" => Ok(GeneratedField::IntervalMonthDayNano), @@ -6595,6 +6638,12 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { } value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::LargeUtf8Value); } + GeneratedField::Utf8ViewValue => { + if value__.is_some() { + return Err(serde::de::Error::duplicate_field("utf8ViewValue")); + } + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Utf8ViewValue); + } GeneratedField::Int8Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("int8Value")); @@ -6772,6 +6821,12 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { } value__ = map_.next_value::<::std::option::Option<::pbjson::private::BytesDeserialize<_>>>()?.map(|x| scalar_value::Value::LargeBinaryValue(x.0)); } + GeneratedField::BinaryViewValue => { + if value__.is_some() { + return Err(serde::de::Error::duplicate_field("binaryViewValue")); + } + value__ = map_.next_value::<::std::option::Option<::pbjson::private::BytesDeserialize<_>>>()?.map(|x| scalar_value::Value::BinaryViewValue(x.0)); + } GeneratedField::Time64Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time64Value")); diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index ff17a40738b5..b306f3212a2f 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -326,7 +326,7 @@ pub struct ScalarFixedSizeBinary { pub struct ScalarValue { #[prost( oneof = "scalar_value::Value", - tags = "33, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 32, 20, 39, 21, 24, 35, 36, 37, 38, 26, 27, 28, 29, 30, 25, 31, 34, 42" + tags = "33, 1, 2, 3, 23, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 32, 20, 39, 21, 24, 35, 36, 37, 38, 26, 27, 28, 29, 22, 30, 25, 31, 34, 42" )] pub value: ::core::option::Option, } @@ -345,6 +345,8 @@ pub mod scalar_value { Utf8Value(::prost::alloc::string::String), #[prost(string, tag = "3")] LargeUtf8Value(::prost::alloc::string::String), + #[prost(string, tag = "23")] + Utf8ViewValue(::prost::alloc::string::String), #[prost(int32, tag = "4")] Int8Value(i32), #[prost(int32, tag = "5")] @@ -402,6 +404,8 @@ pub mod scalar_value { BinaryValue(::prost::alloc::vec::Vec), #[prost(bytes, tag = "29")] LargeBinaryValue(::prost::alloc::vec::Vec), + #[prost(bytes, tag = "22")] + BinaryViewValue(::prost::alloc::vec::Vec), #[prost(message, tag = "30")] Time64Value(super::ScalarTime64Value), #[prost(message, tag = "25")] @@ -440,7 +444,7 @@ pub struct Decimal256 { pub struct ArrowType { #[prost( oneof = "arrow_type::ArrowTypeEnum", - tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 32, 15, 16, 31, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 33" + tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 35, 32, 15, 34, 16, 31, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 33" )] pub arrow_type_enum: ::core::option::Option, } @@ -482,10 +486,14 @@ pub mod arrow_type { Float64(super::EmptyMessage), #[prost(message, tag = "14")] Utf8(super::EmptyMessage), + #[prost(message, tag = "35")] + Utf8View(super::EmptyMessage), #[prost(message, tag = "32")] LargeUtf8(super::EmptyMessage), #[prost(message, tag = "15")] Binary(super::EmptyMessage), + #[prost(message, tag = "34")] + BinaryView(super::EmptyMessage), #[prost(int32, tag = "16")] FixedSizeBinary(i32), #[prost(message, tag = "31")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 8e7ee9a7d6fa..a3dc826a79ca 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -347,6 +347,11 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { Value::LargeUtf8Value(s.to_owned()) }) } + ScalarValue::Utf8View(val) => { + create_proto_scalar(val.as_ref(), &data_type, |s| { + Value::Utf8ViewValue(s.to_owned()) + }) + } ScalarValue::List(arr) => { encode_scalar_nested_value(arr.to_owned() as ArrayRef, val) } @@ -461,6 +466,11 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { Value::BinaryValue(s.to_owned()) }) } + ScalarValue::BinaryView(val) => { + create_proto_scalar(val.as_ref(), &data_type, |s| { + Value::BinaryViewValue(s.to_owned()) + }) + } ScalarValue::LargeBinary(val) => { create_proto_scalar(val.as_ref(), &data_type, |s| { Value::LargeBinaryValue(s.to_owned()) diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index ff17a40738b5..b306f3212a2f 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -326,7 +326,7 @@ pub struct ScalarFixedSizeBinary { pub struct ScalarValue { #[prost( oneof = "scalar_value::Value", - tags = "33, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 32, 20, 39, 21, 24, 35, 36, 37, 38, 26, 27, 28, 29, 30, 25, 31, 34, 42" + tags = "33, 1, 2, 3, 23, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 32, 20, 39, 21, 24, 35, 36, 37, 38, 26, 27, 28, 29, 22, 30, 25, 31, 34, 42" )] pub value: ::core::option::Option, } @@ -345,6 +345,8 @@ pub mod scalar_value { Utf8Value(::prost::alloc::string::String), #[prost(string, tag = "3")] LargeUtf8Value(::prost::alloc::string::String), + #[prost(string, tag = "23")] + Utf8ViewValue(::prost::alloc::string::String), #[prost(int32, tag = "4")] Int8Value(i32), #[prost(int32, tag = "5")] @@ -402,6 +404,8 @@ pub mod scalar_value { BinaryValue(::prost::alloc::vec::Vec), #[prost(bytes, tag = "29")] LargeBinaryValue(::prost::alloc::vec::Vec), + #[prost(bytes, tag = "22")] + BinaryViewValue(::prost::alloc::vec::Vec), #[prost(message, tag = "30")] Time64Value(super::ScalarTime64Value), #[prost(message, tag = "25")] @@ -440,7 +444,7 @@ pub struct Decimal256 { pub struct ArrowType { #[prost( oneof = "arrow_type::ArrowTypeEnum", - tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 32, 15, 16, 31, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 33" + tags = "1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 35, 32, 15, 34, 16, 31, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 33" )] pub arrow_type_enum: ::core::option::Option, } @@ -482,10 +486,14 @@ pub mod arrow_type { Float64(super::EmptyMessage), #[prost(message, tag = "14")] Utf8(super::EmptyMessage), + #[prost(message, tag = "35")] + Utf8View(super::EmptyMessage), #[prost(message, tag = "32")] LargeUtf8(super::EmptyMessage), #[prost(message, tag = "15")] Binary(super::EmptyMessage), + #[prost(message, tag = "34")] + BinaryView(super::EmptyMessage), #[prost(int32, tag = "16")] FixedSizeBinary(i32), #[prost(message, tag = "31")] diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 12c48054f1a7..1d197f3a0d8a 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -709,12 +709,20 @@ impl Unparser<'_> { ast::Value::SingleQuotedString(str.to_string()), )), ScalarValue::Utf8(None) => Ok(ast::Expr::Value(ast::Value::Null)), + ScalarValue::Utf8View(Some(str)) => Ok(ast::Expr::Value( + ast::Value::SingleQuotedString(str.to_string()), + )), + ScalarValue::Utf8View(None) => Ok(ast::Expr::Value(ast::Value::Null)), ScalarValue::LargeUtf8(Some(str)) => Ok(ast::Expr::Value( ast::Value::SingleQuotedString(str.to_string()), )), ScalarValue::LargeUtf8(None) => Ok(ast::Expr::Value(ast::Value::Null)), ScalarValue::Binary(Some(_)) => not_impl_err!("Unsupported scalar: {v:?}"), ScalarValue::Binary(None) => Ok(ast::Expr::Value(ast::Value::Null)), + ScalarValue::BinaryView(Some(_)) => { + not_impl_err!("Unsupported scalar: {v:?}") + } + ScalarValue::BinaryView(None) => Ok(ast::Expr::Value(ast::Value::Null)), ScalarValue::FixedSizeBinary(..) => { not_impl_err!("Unsupported scalar: {v:?}") } diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index c928b96e0321..ab4ff9e2ce92 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -422,3 +422,13 @@ query ? select arrow_cast([1, 2, 3], 'FixedSizeList(3, Int64)'); ---- [1, 2, 3] + +# Tests for Utf8View +query ?T +select arrow_cast('MyAwesomeString', 'Utf8View'), arrow_typeof(arrow_cast('MyAwesomeString', 'Utf8View')) +---- +MyAwesomeString Utf8View + +# Fails until we update arrow-rs with support for https://github.com/apache/arrow-rs/pull/5894 +query error DataFusion error: SQL error: ParserError\("Expected an SQL statement, found: arrow_cast"\) +arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) From a8847e1a825f8e588362430d99363672d8c3e7db Mon Sep 17 00:00:00 2001 From: Andrey Koshchiy Date: Mon, 17 Jun 2024 23:28:12 +0300 Subject: [PATCH 10/54] fix: Support `NOT IN ()` via anti join (#10936) * fix: rewriting NOT IN () to anti join * add wrapped_not_not_in_subquery test --- .../src/decorrelate_predicate_subquery.rs | 86 +++++++++++++++++-- .../sqllogictest/test_files/subquery.slt | 13 +++ 2 files changed, 94 insertions(+), 5 deletions(-) diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 88ce300e5c9a..e5e97b693c6a 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -33,7 +33,7 @@ use datafusion_expr::expr_rewriter::create_col_from_scalar_expr; use datafusion_expr::logical_plan::{JoinType, Subquery}; use datafusion_expr::utils::{conjunction, split_conjunction, split_conjunction_owned}; use datafusion_expr::{ - exists, in_subquery, not_exists, not_in_subquery, BinaryExpr, Expr, Filter, + exists, in_subquery, not, not_exists, not_in_subquery, BinaryExpr, Expr, Filter, LogicalPlan, LogicalPlanBuilder, Operator, }; @@ -79,6 +79,25 @@ impl DecorrelatePredicateSubquery { let mut others = vec![]; for it in filters.into_iter() { match it { + Expr::Not(not_expr) => match *not_expr { + Expr::InSubquery(InSubquery { + expr, + subquery, + negated, + }) => { + let new_subquery = self.rewrite_subquery(subquery, config)?; + subqueries.push(SubqueryInfo::new_with_in_expr( + new_subquery, + *expr, + !negated, + )); + } + Expr::Exists(Exists { subquery, negated }) => { + let new_subquery = self.rewrite_subquery(subquery, config)?; + subqueries.push(SubqueryInfo::new(new_subquery, !negated)); + } + expr => others.push(not(expr)), + }, Expr::InSubquery(InSubquery { expr, subquery, @@ -126,9 +145,17 @@ impl OptimizerRule for DecorrelatePredicateSubquery { }; // if there are no subqueries in the predicate, return the original plan - let has_subqueries = split_conjunction(&filter.predicate) - .iter() - .any(|expr| matches!(expr, Expr::InSubquery(_) | Expr::Exists(_))); + let has_subqueries = + split_conjunction(&filter.predicate) + .iter() + .any(|expr| match expr { + Expr::Not(not_expr) => { + matches!(not_expr.as_ref(), Expr::InSubquery(_) | Expr::Exists(_)) + } + Expr::InSubquery(_) | Expr::Exists(_) => true, + _ => false, + }); + if !has_subqueries { return Ok(Transformed::no(LogicalPlan::Filter(filter))); } @@ -351,7 +378,7 @@ mod tests { use crate::test::*; use arrow::datatypes::DataType; - use datafusion_expr::{and, binary_expr, col, lit, or, out_ref_col}; + use datafusion_expr::{and, binary_expr, col, lit, not, or, out_ref_col}; fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_display_indent( @@ -1099,6 +1126,55 @@ mod tests { Ok(()) } + #[test] + fn wrapped_not_in_subquery() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .filter(not(in_subquery(col("c"), test_subquery_with_name("sq")?)))? + .project(vec![col("test.b")])? + .build()?; + + let expected = "Projection: test.b [b:UInt32]\ + \n LeftAnti Join: Filter: test.c = __correlated_sq_1.c [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [c:UInt32]\ + \n Projection: sq.c [c:UInt32]\ + \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_eq_display_indent( + Arc::new(DecorrelatePredicateSubquery::new()), + plan, + expected, + ); + Ok(()) + } + + #[test] + fn wrapped_not_not_in_subquery() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .filter(not(not_in_subquery( + col("c"), + test_subquery_with_name("sq")?, + )))? + .project(vec![col("test.b")])? + .build()?; + + let expected = "Projection: test.b [b:UInt32]\ + \n LeftSemi Join: Filter: test.c = __correlated_sq_1.c [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [c:UInt32]\ + \n Projection: sq.c [c:UInt32]\ + \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_eq_display_indent( + Arc::new(DecorrelatePredicateSubquery::new()), + plan, + expected, + ); + Ok(()) + } + #[test] fn in_subquery_both_side_expr() -> Result<()> { let table_scan = test_table_scan()?; diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 73fc30c9ae5d..eb0904b2308b 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -127,6 +127,19 @@ where t1.t1_id + 12 not in ( ---- 22 b 2 +# wrapped_not_in_subquery_to_join_with_correlated_outer_filter +query ITI rowsort +select t1.t1_id, + t1.t1_name, + t1.t1_int +from t1 +where not t1.t1_id + 12 in ( + select t2.t2_id + 1 from t2 where t1.t1_int > 0 + ) +---- +22 b 2 + + # in subquery with two parentheses, see #5529 query ITI rowsort select t1.t1_id, From ac161bba336d098eab46f666af4664de7e8cd29f Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 18 Jun 2024 13:04:26 +0800 Subject: [PATCH 11/54] fix: CTEs defined in a subquery can escape their scope (#10954) * fix: CTEs defined in a subquery can escape their scope * Add test --- datafusion/sql/src/cte.rs | 11 +++----- datafusion/sql/src/expr/subquery.rs | 6 ++--- datafusion/sql/src/planner.rs | 23 ++++++++++------ datafusion/sql/src/query.rs | 18 +++++-------- datafusion/sql/src/relation/join.rs | 31 +++------------------- datafusion/sqllogictest/test_files/cte.slt | 26 +++++++++++++++++- 6 files changed, 56 insertions(+), 59 deletions(-) diff --git a/datafusion/sql/src/cte.rs b/datafusion/sql/src/cte.rs index 4f7b9bb6d11d..0035dcda6ed7 100644 --- a/datafusion/sql/src/cte.rs +++ b/datafusion/sql/src/cte.rs @@ -66,10 +66,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { cte_query: Query, planner_context: &mut PlannerContext, ) -> Result { - // CTE expr don't need extend outer_query_schema, - // so we clone a new planner_context here. - let mut cte_planner_context = planner_context.clone(); - self.query_to_plan(cte_query, &mut cte_planner_context) + self.query_to_plan(cte_query, planner_context) } fn recursive_cte( @@ -113,8 +110,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // allow us to infer the schema to be used in the recursive term. // ---------- Step 1: Compile the static term ------------------ - let static_plan = - self.set_expr_to_plan(*left_expr, &mut planner_context.clone())?; + let static_plan = self.set_expr_to_plan(*left_expr, planner_context)?; // Since the recursive CTEs include a component that references a // table with its name, like the example below: @@ -166,8 +162,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // this uses the named_relation we inserted above to resolve the // relation. This ensures that the recursive term uses the named relation logical plan // and thus the 'continuance' physical plan as its input and source - let recursive_plan = - self.set_expr_to_plan(*right_expr, &mut planner_context.clone())?; + let recursive_plan = self.set_expr_to_plan(*right_expr, planner_context)?; // Check if the recursive term references the CTE itself, // if not, it is a non-recursive CTE diff --git a/datafusion/sql/src/expr/subquery.rs b/datafusion/sql/src/expr/subquery.rs index d34065d92fe5..ff161c6ed644 100644 --- a/datafusion/sql/src/expr/subquery.rs +++ b/datafusion/sql/src/expr/subquery.rs @@ -33,7 +33,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, ) -> Result { let old_outer_query_schema = - planner_context.set_outer_query_schema(Some(input_schema.clone())); + planner_context.set_outer_query_schema(Some(input_schema.clone().into())); let sub_plan = self.query_to_plan(subquery, planner_context)?; let outer_ref_columns = sub_plan.all_out_ref_exprs(); planner_context.set_outer_query_schema(old_outer_query_schema); @@ -55,7 +55,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, ) -> Result { let old_outer_query_schema = - planner_context.set_outer_query_schema(Some(input_schema.clone())); + planner_context.set_outer_query_schema(Some(input_schema.clone().into())); let sub_plan = self.query_to_plan(subquery, planner_context)?; let outer_ref_columns = sub_plan.all_out_ref_exprs(); planner_context.set_outer_query_schema(old_outer_query_schema); @@ -77,7 +77,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, ) -> Result { let old_outer_query_schema = - planner_context.set_outer_query_schema(Some(input_schema.clone())); + planner_context.set_outer_query_schema(Some(input_schema.clone().into())); let sub_plan = self.query_to_plan(subquery, planner_context)?; let outer_ref_columns = sub_plan.all_out_ref_exprs(); planner_context.set_outer_query_schema(old_outer_query_schema); diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index a92e64597e82..30f95170a34f 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -22,7 +22,7 @@ use std::vec; use arrow_schema::*; use datafusion_common::{ - field_not_found, internal_err, plan_datafusion_err, SchemaError, + field_not_found, internal_err, plan_datafusion_err, DFSchemaRef, SchemaError, }; use datafusion_expr::WindowUDF; use sqlparser::ast::TimezoneInfo; @@ -139,16 +139,23 @@ impl IdentNormalizer { /// Common Table Expression (CTE) provided with WITH clause and /// Parameter Data Types provided with PREPARE statement and the query schema of the /// outer query plan +/// +/// # Cloning +/// +/// Only the `ctes` are truly cloned when the `PlannerContext` is cloned. This helps resolve +/// scoping issues of CTEs. By using cloning, a subquery can inherit CTEs from the outer query +/// and can also define its own private CTEs without affecting the outer query. +/// #[derive(Debug, Clone)] pub struct PlannerContext { /// Data types for numbered parameters ($1, $2, etc), if supplied /// in `PREPARE` statement - prepare_param_data_types: Vec, + prepare_param_data_types: Arc>, /// Map of CTE name to logical plan of the WITH clause. /// Use `Arc` to allow cheap cloning ctes: HashMap>, /// The query schema of the outer query plan, used to resolve the columns in subquery - outer_query_schema: Option, + outer_query_schema: Option, } impl Default for PlannerContext { @@ -161,7 +168,7 @@ impl PlannerContext { /// Create an empty PlannerContext pub fn new() -> Self { Self { - prepare_param_data_types: vec![], + prepare_param_data_types: Arc::new(vec![]), ctes: HashMap::new(), outer_query_schema: None, } @@ -172,21 +179,21 @@ impl PlannerContext { mut self, prepare_param_data_types: Vec, ) -> Self { - self.prepare_param_data_types = prepare_param_data_types; + self.prepare_param_data_types = prepare_param_data_types.into(); self } // return a reference to the outer queries schema pub fn outer_query_schema(&self) -> Option<&DFSchema> { - self.outer_query_schema.as_ref() + self.outer_query_schema.as_ref().map(|s| s.as_ref()) } /// sets the outer query schema, returning the existing one, if /// any pub fn set_outer_query_schema( &mut self, - mut schema: Option, - ) -> Option { + mut schema: Option, + ) -> Option { std::mem::swap(&mut self.outer_query_schema, &mut schema); schema } diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index fdc739646a46..cbbff19321d8 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -29,23 +29,17 @@ use sqlparser::ast::{ }; impl<'a, S: ContextProvider> SqlToRel<'a, S> { - /// Generate a logical plan from an SQL query + /// Generate a logical plan from an SQL query/subquery pub(crate) fn query_to_plan( &self, query: Query, - planner_context: &mut PlannerContext, + outer_planner_context: &mut PlannerContext, ) -> Result { - self.query_to_plan_with_schema(query, planner_context) - } + // Each query has its own planner context, including CTEs that are visible within that query. + // It also inherits the CTEs from the outer query by cloning the outer planner context. + let mut query_plan_context = outer_planner_context.clone(); + let planner_context = &mut query_plan_context; - /// Generate a logic plan from an SQL query. - /// It's implementation of `subquery_to_plan` and `query_to_plan`. - /// It shouldn't be invoked directly. - fn query_to_plan_with_schema( - &self, - query: Query, - planner_context: &mut PlannerContext, - ) -> Result { if let Some(with) = query.with { self.plan_with_clause(with, planner_context)?; } diff --git a/datafusion/sql/src/relation/join.rs b/datafusion/sql/src/relation/join.rs index 262bae397cee..ee2e35b550f6 100644 --- a/datafusion/sql/src/relation/join.rs +++ b/datafusion/sql/src/relation/join.rs @@ -27,34 +27,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { t: TableWithJoins, planner_context: &mut PlannerContext, ) -> Result { - // From clause may exist CTEs, we should separate them from global CTEs. - // CTEs in from clause are allowed to be duplicated. - // Such as `select * from (WITH source AS (select 1 as e) SELECT * FROM source) t1, (WITH source AS (select 1 as e) SELECT * FROM source) t2;` which is valid. - // So always use original global CTEs to plan CTEs in from clause. - // Btw, don't need to add CTEs in from to global CTEs. - let origin_planner_context = planner_context.clone(); - let left = self.create_relation(t.relation, planner_context)?; - match t.joins.len() { - 0 => { - *planner_context = origin_planner_context; - Ok(left) - } - _ => { - let mut joins = t.joins.into_iter(); - *planner_context = origin_planner_context.clone(); - let mut left = self.parse_relation_join( - left, - joins.next().unwrap(), // length of joins > 0 - planner_context, - )?; - for join in joins { - *planner_context = origin_planner_context.clone(); - left = self.parse_relation_join(left, join, planner_context)?; - } - *planner_context = origin_planner_context; - Ok(left) - } + let mut left = self.create_relation(t.relation, planner_context)?; + for join in t.joins.into_iter() { + left = self.parse_relation_join(left, join, planner_context)?; } + Ok(left) } fn parse_relation_join( diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index d8eaa51fc88a..e9fcf07e7739 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -834,4 +834,28 @@ SELECT * FROM non_recursive_cte, recursive_cte; query I WITH t AS (SELECT * FROM t where t.a < 2) SELECT * FROM t ---- -1 \ No newline at end of file +1 + +# Issue: https://github.com/apache/datafusion/issues/10914 +# The CTE defined within the subquery is only visible inside that subquery. +query I rowsort +(WITH t AS (SELECT 400) SELECT * FROM t) UNION (SELECT * FROM t); +---- +1 +2 +3 +400 + +query error DataFusion error: Error during planning: table 'datafusion\.public\.cte' not found +(WITH cte AS (SELECT 400) SELECT * FROM cte) UNION (SELECT * FROM cte); + +# Test duplicate CTE names in different subqueries in the FROM clause. +query III rowsort +SELECT * FROM + (WITH t AS (select 400 as e) SELECT * FROM t) t1, + (WITH t AS (select 500 as e) SELECT * FROM t) t2, + t +---- +400 500 1 +400 500 2 +400 500 3 From 0c177d18dc57fe81a7a23e1b1a41991dc88bb4f6 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Tue, 18 Jun 2024 12:25:28 +0200 Subject: [PATCH 12/54] Use shorter aliases in CSE (#10939) * initial change * test renaming * use counter instead of indexmap * order slt tests * change cse tests * restore slt tests * fix slt test * formatting * ensure no alias collision * keep original alias numbers for collision * ensure no collision in aggregate cse * use `AliasGenerator` to generate aliases, use `__cse` prefix in common expression aliases, remove `DataType` from `ExprStats` as not needed, store aliases in `CommonExprs`, revert unnecessary changes * use `into_values()` instead of `into_iter()` where possible * fix docstring of `ExprStats` and `CommonExprs` * use `__common_expr` prefix --------- Co-authored-by: Mohamed Abdeen --- .../optimizer/src/common_subexpr_eliminate.rs | 368 +++++++++--------- .../sqllogictest/test_files/group_by.slt | 8 +- datafusion/sqllogictest/test_files/select.slt | 16 +- .../sqllogictest/test_files/subquery.slt | 8 +- .../sqllogictest/test_files/tpch/q1.slt.part | 6 +- datafusion/sqllogictest/test_files/window.slt | 126 +++--- 6 files changed, 271 insertions(+), 261 deletions(-) diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 3ed1309f1544..e150a957bfcf 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -22,19 +22,19 @@ use std::sync::Arc; use crate::{utils, OptimizerConfig, OptimizerRule}; -use arrow::datatypes::{DataType, Field}; +use datafusion_common::alias::AliasGenerator; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter, TreeNodeVisitor, }; -use datafusion_common::{ - internal_err, qualified_name, Column, DFSchema, DFSchemaRef, DataFusionError, Result, -}; +use datafusion_common::{qualified_name, Column, DFSchema, DataFusionError, Result}; use datafusion_expr::expr::Alias; use datafusion_expr::logical_plan::{Aggregate, LogicalPlan, Projection, Window}; use datafusion_expr::{col, Expr, ExprSchemable}; use indexmap::IndexMap; +const CSE_PREFIX: &str = "__common_expr"; + /// Identifier that represents a subexpression tree. /// /// Note that the current implementation contains: @@ -79,16 +79,12 @@ type Identifier = String; /// ``` type IdArray = Vec<(usize, Identifier)>; -/// A map that contains statistics of expressions by their identifiers. -/// It contains: -/// - The number of occurrences and -/// - The DataType -/// of an expression. -type ExprStats = HashMap; +/// A map that contains the number of occurrences of expressions by their identifiers. +type ExprStats = HashMap; -/// A map that contains the common expressions extracted during the second, rewriting -/// traversal. -type CommonExprs = IndexMap; +/// A map that contains the common expressions and their alias extracted during the +/// second, rewriting traversal. +type CommonExprs = IndexMap; /// Performs Common Sub-expression Elimination optimization. /// @@ -131,6 +127,7 @@ impl CommonSubexprEliminate { arrays_list: &[&[IdArray]], expr_stats: &ExprStats, common_exprs: &mut CommonExprs, + alias_generator: &AliasGenerator, ) -> Result>> { exprs_list .iter() @@ -141,7 +138,13 @@ impl CommonSubexprEliminate { .cloned() .zip(arrays.iter()) .map(|(expr, id_array)| { - replace_common_expr(expr, id_array, expr_stats, common_exprs) + replace_common_expr( + expr, + id_array, + expr_stats, + common_exprs, + alias_generator, + ) }) .collect::>>() }) @@ -164,21 +167,21 @@ impl CommonSubexprEliminate { expr_stats: &ExprStats, config: &dyn OptimizerConfig, ) -> Result<(Vec>, LogicalPlan)> { - let mut common_exprs = IndexMap::new(); - + let mut common_exprs = CommonExprs::new(); let rewrite_exprs = self.rewrite_exprs_list( exprs_list, arrays_list, expr_stats, &mut common_exprs, + &config.alias_generator(), )?; let mut new_input = self .try_optimize(input, config)? .unwrap_or_else(|| input.clone()); + if !common_exprs.is_empty() { - new_input = - build_common_expr_project_plan(new_input, common_exprs, expr_stats)?; + new_input = build_common_expr_project_plan(new_input, common_exprs)?; } Ok((rewrite_exprs, new_input)) @@ -209,13 +212,7 @@ impl CommonSubexprEliminate { } = window; plan = input.as_ref().clone(); - let input_schema = Arc::clone(input.schema()); - let arrays = to_arrays( - &window_expr, - input_schema, - &mut expr_stats, - ExprMask::Normal, - )?; + let arrays = to_arrays(&window_expr, &mut expr_stats, ExprMask::Normal)?; window_exprs.push(window_expr); arrays_per_window.push(arrays); @@ -277,15 +274,8 @@ impl CommonSubexprEliminate { let mut expr_stats = ExprStats::new(); // rewrite inputs - let input_schema = Arc::clone(input.schema()); - let group_arrays = to_arrays( - group_expr, - Arc::clone(&input_schema), - &mut expr_stats, - ExprMask::Normal, - )?; - let aggr_arrays = - to_arrays(aggr_expr, input_schema, &mut expr_stats, ExprMask::Normal)?; + let group_arrays = to_arrays(group_expr, &mut expr_stats, ExprMask::Normal)?; + let aggr_arrays = to_arrays(aggr_expr, &mut expr_stats, ExprMask::Normal)?; let (mut new_expr, new_input) = self.rewrite_expr( &[group_expr, aggr_expr], @@ -303,16 +293,16 @@ impl CommonSubexprEliminate { let new_input_schema = Arc::clone(new_input.schema()); let aggr_arrays = to_arrays( &new_aggr_expr, - new_input_schema.clone(), &mut expr_stats, ExprMask::NormalAndAggregates, )?; - let mut common_exprs = IndexMap::new(); + let mut common_exprs = CommonExprs::new(); let mut rewritten = self.rewrite_exprs_list( &[&new_aggr_expr], &[&aggr_arrays], &expr_stats, &mut common_exprs, + &config.alias_generator(), )?; let rewritten = pop_expr(&mut rewritten)?; @@ -330,11 +320,8 @@ impl CommonSubexprEliminate { .map(LogicalPlan::Aggregate) } else { let mut agg_exprs = common_exprs - .into_iter() - .map(|(expr_id, expr)| { - // todo: check `nullable` - expr.alias(expr_id) - }) + .into_values() + .map(|(expr, expr_alias)| expr.alias(expr_alias)) .collect::>(); let mut proj_exprs = vec![]; @@ -347,14 +334,15 @@ impl CommonSubexprEliminate { agg_exprs.push(expr.alias(&name)); proj_exprs.push(Expr::Column(Column::from_name(name))); } else { - let id = expr_identifier(&expr_rewritten, "".to_string()); + let expr_alias = config.alias_generator().next(CSE_PREFIX); let (qualifier, field) = expr_rewritten.to_field(&new_input_schema)?; let out_name = qualified_name(qualifier.as_ref(), field.name()); - agg_exprs.push(expr_rewritten.alias(&id)); - proj_exprs - .push(Expr::Column(Column::from_name(id)).alias(out_name)); + agg_exprs.push(expr_rewritten.alias(&expr_alias)); + proj_exprs.push( + Expr::Column(Column::from_name(expr_alias)).alias(out_name), + ); } } else { proj_exprs.push(expr_rewritten); @@ -382,11 +370,10 @@ impl CommonSubexprEliminate { let expr = plan.expressions(); let inputs = plan.inputs(); let input = inputs[0]; - let input_schema = Arc::clone(input.schema()); let mut expr_stats = ExprStats::new(); // Visit expr list and build expr identifier to occuring count map (`expr_stats`). - let arrays = to_arrays(&expr, input_schema, &mut expr_stats, ExprMask::Normal)?; + let arrays = to_arrays(&expr, &mut expr_stats, ExprMask::Normal)?; let (mut new_expr, new_input) = self.rewrite_expr(&[&expr], &[&arrays], input, &expr_stats, config)?; @@ -477,20 +464,13 @@ fn pop_expr(new_expr: &mut Vec>) -> Result> { fn to_arrays( expr: &[Expr], - input_schema: DFSchemaRef, expr_stats: &mut ExprStats, expr_mask: ExprMask, ) -> Result> { expr.iter() .map(|e| { let mut id_array = vec![]; - expr_to_identifier( - e, - expr_stats, - &mut id_array, - Arc::clone(&input_schema), - expr_mask, - )?; + expr_to_identifier(e, expr_stats, &mut id_array, expr_mask)?; Ok(id_array) }) @@ -510,19 +490,13 @@ fn to_arrays( fn build_common_expr_project_plan( input: LogicalPlan, common_exprs: CommonExprs, - expr_stats: &ExprStats, ) -> Result { let mut fields_set = BTreeSet::new(); let mut project_exprs = common_exprs - .into_iter() - .map(|(expr_id, expr)| { - let Some((_, data_type)) = expr_stats.get(&expr_id) else { - return internal_err!("expr_stats invalid state"); - }; - // todo: check `nullable` - let field = Field::new(&expr_id, data_type.clone(), true); - fields_set.insert(field.name().to_owned()); - Ok(expr.alias(expr_id)) + .into_values() + .map(|(expr, expr_alias)| { + fields_set.insert(expr_alias.clone()); + Ok(expr.alias(expr_alias)) }) .collect::>>()?; @@ -637,9 +611,6 @@ struct ExprIdentifierVisitor<'a> { expr_stats: &'a mut ExprStats, // cache to speed up second traversal id_array: &'a mut IdArray, - // input schema for the node that we're optimizing, so we can determine the correct datatype - // for each subexpression - input_schema: DFSchemaRef, // inner states visit_stack: Vec, // preorder index, start from 0. @@ -714,14 +685,7 @@ impl<'n> TreeNodeVisitor<'n> for ExprIdentifierVisitor<'_> { self.id_array[down_index].0 = self.up_index; if !self.expr_mask.ignores(expr) { self.id_array[down_index].1.clone_from(&expr_id); - - // TODO: can we capture the data type in the second traversal only for - // replaced expressions? - let data_type = expr.get_type(&self.input_schema)?; - let (count, _) = self - .expr_stats - .entry(expr_id.clone()) - .or_insert((0, data_type)); + let count = self.expr_stats.entry(expr_id.clone()).or_insert(0); *count += 1; } self.visit_stack.push(VisitRecord::ExprItem(expr_id)); @@ -740,13 +704,11 @@ fn expr_to_identifier( expr: &Expr, expr_stats: &mut ExprStats, id_array: &mut IdArray, - input_schema: DFSchemaRef, expr_mask: ExprMask, ) -> Result<()> { expr.visit(&mut ExprIdentifierVisitor { expr_stats, id_array, - input_schema, visit_stack: vec![], down_index: 0, up_index: 0, @@ -771,6 +733,8 @@ struct CommonSubexprRewriter<'a> { down_index: usize, // how many aliases have we seen so far alias_counter: usize, + // alias generator for extracted common expressions + alias_generator: &'a AliasGenerator, } impl TreeNodeRewriter for CommonSubexprRewriter<'_> { @@ -780,17 +744,18 @@ impl TreeNodeRewriter for CommonSubexprRewriter<'_> { if matches!(expr, Expr::Alias(_)) { self.alias_counter -= 1 } + Ok(Transformed::no(expr)) } fn f_down(&mut self, expr: Expr) -> Result> { - // The `CommonSubexprRewriter` relies on `ExprIdentifierVisitor` to generate - // the `id_array`, which records the expr's identifier used to rewrite expr. So if we - // skip an expr in `ExprIdentifierVisitor`, we should skip it here, too. if matches!(expr, Expr::Alias(_)) { self.alias_counter += 1; } + // The `CommonSubexprRewriter` relies on `ExprIdentifierVisitor` to generate + // the `id_array`, which records the expr's identifier used to rewrite expr. So if we + // skip an expr in `ExprIdentifierVisitor`, we should skip it here, too. if expr.short_circuits() || expr.is_volatile()? { return Ok(Transformed::new(expr, false, TreeNodeRecursion::Jump)); } @@ -803,8 +768,8 @@ impl TreeNodeRewriter for CommonSubexprRewriter<'_> { return Ok(Transformed::no(expr)); } - let (counter, _) = self.expr_stats.get(expr_id).unwrap(); - if *counter > 1 { + let count = self.expr_stats.get(expr_id).unwrap(); + if *count > 1 { // step index to skip all sub-node (which has smaller series number). while self.down_index < self.id_array.len() && self.id_array[self.down_index].0 < *up_index @@ -813,14 +778,18 @@ impl TreeNodeRewriter for CommonSubexprRewriter<'_> { } let expr_name = expr.display_name()?; - self.common_exprs.insert(expr_id.clone(), expr); + let (_, expr_alias) = + self.common_exprs.entry(expr_id.clone()).or_insert_with(|| { + let expr_alias = self.alias_generator.next(CSE_PREFIX); + (expr, expr_alias) + }); // alias the expressions without an `Alias` ancestor node let rewritten = if self.alias_counter > 0 { - col(expr_id) + col(expr_alias.clone()) } else { self.alias_counter += 1; - col(expr_id).alias(expr_name) + col(expr_alias.clone()).alias(expr_name) }; Ok(Transformed::new(rewritten, true, TreeNodeRecursion::Jump)) @@ -837,6 +806,7 @@ fn replace_common_expr( id_array: &IdArray, expr_stats: &ExprStats, common_exprs: &mut CommonExprs, + alias_generator: &AliasGenerator, ) -> Result { expr.rewrite(&mut CommonSubexprRewriter { expr_stats, @@ -844,6 +814,7 @@ fn replace_common_expr( common_exprs, down_index: 0, alias_counter: 0, + alias_generator, }) .data() } @@ -852,7 +823,7 @@ fn replace_common_expr( mod test { use std::iter; - use arrow::datatypes::Schema; + use arrow::datatypes::{DataType, Field, Schema}; use datafusion_expr::logical_plan::{table_scan, JoinType}; @@ -868,10 +839,16 @@ mod test { use super::*; - fn assert_optimized_plan_eq(expected: &str, plan: &LogicalPlan) { + fn assert_optimized_plan_eq( + expected: &str, + plan: &LogicalPlan, + config: Option<&dyn OptimizerConfig>, + ) { let optimizer = CommonSubexprEliminate {}; + let default_config = OptimizerContext::new(); + let config = config.unwrap_or(&default_config); let optimized_plan = optimizer - .try_optimize(plan, &OptimizerContext::new()) + .try_optimize(plan, config) .unwrap() .expect("failed to optimize plan"); let formatted_plan = format!("{optimized_plan:?}"); @@ -882,24 +859,9 @@ mod test { fn id_array_visitor() -> Result<()> { let expr = ((sum(col("a") + lit(1))) - avg(col("c"))) * lit(2); - let schema = Arc::new(DFSchema::from_unqualifed_fields( - vec![ - Field::new("a", DataType::Int64, false), - Field::new("c", DataType::Int64, false), - ] - .into(), - Default::default(), - )?); - // skip aggregates let mut id_array = vec![]; - expr_to_identifier( - &expr, - &mut HashMap::new(), - &mut id_array, - Arc::clone(&schema), - ExprMask::Normal, - )?; + expr_to_identifier(&expr, &mut HashMap::new(), &mut id_array, ExprMask::Normal)?; let expected = vec![ (8, "{(sum(a + Int32(1)) - AVG(c)) * Int32(2)|{Int32(2)}|{sum(a + Int32(1)) - AVG(c)|{AVG(c)|{c}}|{sum(a + Int32(1))|{a + Int32(1)|{Int32(1)}|{a}}}}}"), @@ -923,7 +885,6 @@ mod test { &expr, &mut HashMap::new(), &mut id_array, - Arc::clone(&schema), ExprMask::NormalAndAggregates, )?; @@ -968,11 +929,11 @@ mod test { )? .build()?; - let expected = "Aggregate: groupBy=[[]], aggr=[[sum({test.a * (Int32(1) - test.b)|{Int32(1) - test.b|{test.b}|{Int32(1)}}|{test.a}} AS test.a * Int32(1) - test.b), sum({test.a * (Int32(1) - test.b)|{Int32(1) - test.b|{test.b}|{Int32(1)}}|{test.a}} AS test.a * Int32(1) - test.b * (Int32(1) + test.c))]]\ - \n Projection: test.a * (Int32(1) - test.b) AS {test.a * (Int32(1) - test.b)|{Int32(1) - test.b|{test.b}|{Int32(1)}}|{test.a}}, test.a, test.b, test.c\ + let expected = "Aggregate: groupBy=[[]], aggr=[[sum(__common_expr_1 AS test.a * Int32(1) - test.b), sum(__common_expr_1 AS test.a * Int32(1) - test.b * (Int32(1) + test.c))]]\ + \n Projection: test.a * (Int32(1) - test.b) AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } @@ -988,11 +949,11 @@ mod test { ])? .build()?; - let expected = "Projection: {test.a + test.b|{test.b}|{test.a}} - test.c AS alias1 * {test.a + test.b|{test.b}|{test.a}} AS test.a + test.b, {test.a + test.b|{test.b}|{test.a}} AS test.a + test.b\ - \n Projection: test.a + test.b AS {test.a + test.b|{test.b}|{test.a}}, test.a, test.b, test.c\ + let expected = "Projection: __common_expr_1 - test.c AS alias1 * __common_expr_1 AS test.a + test.b, __common_expr_1 AS test.a + test.b\ + \n Projection: test.a + test.b AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } @@ -1041,11 +1002,11 @@ mod test { )? .build()?; - let expected = "Projection: {AVG(test.a)|{test.a}} AS col1, {AVG(test.a)|{test.a}} AS col2, col3, {AVG(test.c)} AS AVG(test.c), {my_agg(test.a)|{test.a}} AS col4, {my_agg(test.a)|{test.a}} AS col5, col6, {my_agg(test.c)} AS my_agg(test.c)\ - \n Aggregate: groupBy=[[]], aggr=[[AVG(test.a) AS {AVG(test.a)|{test.a}}, my_agg(test.a) AS {my_agg(test.a)|{test.a}}, AVG(test.b) AS col3, AVG(test.c) AS {AVG(test.c)}, my_agg(test.b) AS col6, my_agg(test.c) AS {my_agg(test.c)}]]\ + let expected = "Projection: __common_expr_1 AS col1, __common_expr_1 AS col2, col3, __common_expr_3 AS AVG(test.c), __common_expr_2 AS col4, __common_expr_2 AS col5, col6, __common_expr_4 AS my_agg(test.c)\ + \n Aggregate: groupBy=[[]], aggr=[[AVG(test.a) AS __common_expr_1, my_agg(test.a) AS __common_expr_2, AVG(test.b) AS col3, AVG(test.c) AS __common_expr_3, my_agg(test.b) AS col6, my_agg(test.c) AS __common_expr_4]]\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); // test: trafo after aggregate let plan = LogicalPlanBuilder::from(table_scan.clone()) @@ -1060,11 +1021,11 @@ mod test { )? .build()?; - let expected = "Projection: Int32(1) + {AVG(test.a)|{test.a}} AS AVG(test.a), Int32(1) - {AVG(test.a)|{test.a}} AS AVG(test.a), Int32(1) + {my_agg(test.a)|{test.a}} AS my_agg(test.a), Int32(1) - {my_agg(test.a)|{test.a}} AS my_agg(test.a)\ - \n Aggregate: groupBy=[[]], aggr=[[AVG(test.a) AS {AVG(test.a)|{test.a}}, my_agg(test.a) AS {my_agg(test.a)|{test.a}}]]\ + let expected = "Projection: Int32(1) + __common_expr_1 AS AVG(test.a), Int32(1) - __common_expr_1 AS AVG(test.a), Int32(1) + __common_expr_2 AS my_agg(test.a), Int32(1) - __common_expr_2 AS my_agg(test.a)\ + \n Aggregate: groupBy=[[]], aggr=[[AVG(test.a) AS __common_expr_1, my_agg(test.a) AS __common_expr_2]]\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); // test: transformation before aggregate let plan = LogicalPlanBuilder::from(table_scan.clone()) @@ -1077,9 +1038,11 @@ mod test { )? .build()?; - let expected = "Aggregate: groupBy=[[]], aggr=[[AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}}) AS col1, my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}}) AS col2]]\n Projection: UInt32(1) + test.a AS {UInt32(1) + test.a|{test.a}|{UInt32(1)}}, test.a, test.b, test.c\n TableScan: test"; + let expected ="Aggregate: groupBy=[[]], aggr=[[AVG(__common_expr_1) AS col1, my_agg(__common_expr_1) AS col2]]\ + \n Projection: UInt32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ + \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); // test: common between agg and group let plan = LogicalPlanBuilder::from(table_scan.clone()) @@ -1092,11 +1055,11 @@ mod test { )? .build()?; - let expected = "Aggregate: groupBy=[[{UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a]], aggr=[[AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}}) AS col1, my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}}) AS col2]]\ - \n Projection: UInt32(1) + test.a AS {UInt32(1) + test.a|{test.a}|{UInt32(1)}}, test.a, test.b, test.c\ + let expected = "Aggregate: groupBy=[[__common_expr_1 AS UInt32(1) + test.a]], aggr=[[AVG(__common_expr_1) AS col1, my_agg(__common_expr_1) AS col2]]\ + \n Projection: UInt32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); // test: all mixed let plan = LogicalPlanBuilder::from(table_scan) @@ -1113,18 +1076,18 @@ mod test { )? .build()?; - let expected = "Projection: UInt32(1) + test.a, UInt32(1) + {AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}})|{{UInt32(1) + test.a|{test.a}|{UInt32(1)}}}} AS col1, UInt32(1) - {AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}})|{{UInt32(1) + test.a|{test.a}|{UInt32(1)}}}} AS col2, {AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a)} AS AVG(UInt32(1) + test.a), UInt32(1) + {my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}})|{{UInt32(1) + test.a|{test.a}|{UInt32(1)}}}} AS col3, UInt32(1) - {my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}})|{{UInt32(1) + test.a|{test.a}|{UInt32(1)}}}} AS col4, {my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a)} AS my_agg(UInt32(1) + test.a)\ - \n Aggregate: groupBy=[[{UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a]], aggr=[[AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}}) AS {AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}})|{{UInt32(1) + test.a|{test.a}|{UInt32(1)}}}}, my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}}) AS {my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}})|{{UInt32(1) + test.a|{test.a}|{UInt32(1)}}}}, AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a) AS {AVG({UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a)}, my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a) AS {my_agg({UInt32(1) + test.a|{test.a}|{UInt32(1)}} AS UInt32(1) + test.a)}]]\ - \n Projection: UInt32(1) + test.a AS {UInt32(1) + test.a|{test.a}|{UInt32(1)}}, test.a, test.b, test.c\ + let expected = "Projection: UInt32(1) + test.a, UInt32(1) + __common_expr_2 AS col1, UInt32(1) - __common_expr_2 AS col2, __common_expr_4 AS AVG(UInt32(1) + test.a), UInt32(1) + __common_expr_3 AS col3, UInt32(1) - __common_expr_3 AS col4, __common_expr_5 AS my_agg(UInt32(1) + test.a)\ + \n Aggregate: groupBy=[[__common_expr_1 AS UInt32(1) + test.a]], aggr=[[AVG(__common_expr_1) AS __common_expr_2, my_agg(__common_expr_1) AS __common_expr_3, AVG(__common_expr_1 AS UInt32(1) + test.a) AS __common_expr_4, my_agg(__common_expr_1 AS UInt32(1) + test.a) AS __common_expr_5]]\ + \n Projection: UInt32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } #[test] - fn aggregate_with_releations_and_dots() -> Result<()> { + fn aggregate_with_relations_and_dots() -> Result<()> { let schema = Schema::new(vec![Field::new("col.a", DataType::UInt32, false)]); let table_scan = table_scan(Some("table.test"), &schema, None)?.build()?; @@ -1140,12 +1103,12 @@ mod test { )? .build()?; - let expected = "Projection: table.test.col.a, UInt32(1) + {AVG({UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a)|{{UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a|{{UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}}}}} AS AVG(UInt32(1) + table.test.col.a), {AVG({UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a)|{{UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a|{{UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}}}}} AS AVG(UInt32(1) + table.test.col.a)\ - \n Aggregate: groupBy=[[table.test.col.a]], aggr=[[AVG({UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a) AS {AVG({UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a)|{{UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}} AS UInt32(1) + table.test.col.a|{{UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}}}}}]]\ - \n Projection: UInt32(1) + table.test.col.a AS {UInt32(1) + table.test.col.a|{table.test.col.a}|{UInt32(1)}}, table.test.col.a\ + let expected = "Projection: table.test.col.a, UInt32(1) + __common_expr_2 AS AVG(UInt32(1) + table.test.col.a), __common_expr_2 AS AVG(UInt32(1) + table.test.col.a)\ + \n Aggregate: groupBy=[[table.test.col.a]], aggr=[[AVG(__common_expr_1 AS UInt32(1) + table.test.col.a) AS __common_expr_2]]\ + \n Projection: UInt32(1) + table.test.col.a AS __common_expr_1, table.test.col.a\ \n TableScan: table.test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } @@ -1161,11 +1124,11 @@ mod test { ])? .build()?; - let expected = "Projection: {Int32(1) + test.a|{test.a}|{Int32(1)}} AS first, {Int32(1) + test.a|{test.a}|{Int32(1)}} AS second\ - \n Projection: Int32(1) + test.a AS {Int32(1) + test.a|{test.a}|{Int32(1)}}, test.a, test.b, test.c\ + let expected = "Projection: __common_expr_1 AS first, __common_expr_1 AS second\ + \n Projection: Int32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } @@ -1181,7 +1144,7 @@ mod test { let expected = "Projection: Int32(1) + test.a, test.a + Int32(1)\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } @@ -1199,35 +1162,35 @@ mod test { \n Projection: Int32(1) + test.a, test.a\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } #[test] fn redundant_project_fields() { let table_scan = test_table_scan().unwrap(); - let expr_stats_1 = ExprStats::from([ - ("c+a".to_string(), (1, DataType::UInt32)), - ("b+a".to_string(), (1, DataType::UInt32)), - ]); - let common_exprs_1 = IndexMap::from([ - ("c+a".to_string(), col("c") + col("a")), - ("b+a".to_string(), col("b") + col("a")), - ]); - let exprs_stats_2 = ExprStats::from([ - ("c+a".to_string(), (1, DataType::UInt32)), - ("b+a".to_string(), (1, DataType::UInt32)), + let common_exprs_1 = CommonExprs::from([ + ( + "c+a".to_string(), + (col("c") + col("a"), format!("{CSE_PREFIX}_1")), + ), + ( + "b+a".to_string(), + (col("b") + col("a"), format!("{CSE_PREFIX}_2")), + ), ]); - let common_exprs_2 = IndexMap::from([ - ("c+a".to_string(), col("c+a")), - ("b+a".to_string(), col("b+a")), + let common_exprs_2 = CommonExprs::from([ + ( + "c+a".to_string(), + (col(format!("{CSE_PREFIX}_1")), format!("{CSE_PREFIX}_3")), + ), + ( + "b+a".to_string(), + (col(format!("{CSE_PREFIX}_2")), format!("{CSE_PREFIX}_4")), + ), ]); - let project = - build_common_expr_project_plan(table_scan, common_exprs_1, &expr_stats_1) - .unwrap(); - let project_2 = - build_common_expr_project_plan(project, common_exprs_2, &exprs_stats_2) - .unwrap(); + let project = build_common_expr_project_plan(table_scan, common_exprs_1).unwrap(); + let project_2 = build_common_expr_project_plan(project, common_exprs_2).unwrap(); let mut field_set = BTreeSet::new(); for name in project_2.schema().field_names() { @@ -1244,33 +1207,28 @@ mod test { .unwrap() .build() .unwrap(); - let expr_stats_1 = ExprStats::from([ - ("test1.c+test1.a".to_string(), (1, DataType::UInt32)), - ("test1.b+test1.a".to_string(), (1, DataType::UInt32)), - ]); - let common_exprs_1 = IndexMap::from([ + let common_exprs_1 = CommonExprs::from([ ( "test1.c+test1.a".to_string(), - col("test1.c") + col("test1.a"), + (col("test1.c") + col("test1.a"), format!("{CSE_PREFIX}_1")), ), ( "test1.b+test1.a".to_string(), - col("test1.b") + col("test1.a"), + (col("test1.b") + col("test1.a"), format!("{CSE_PREFIX}_2")), ), ]); - let expr_stats_2 = ExprStats::from([ - ("test1.c+test1.a".to_string(), (1, DataType::UInt32)), - ("test1.b+test1.a".to_string(), (1, DataType::UInt32)), - ]); - let common_exprs_2 = IndexMap::from([ - ("test1.c+test1.a".to_string(), col("test1.c+test1.a")), - ("test1.b+test1.a".to_string(), col("test1.b+test1.a")), + let common_exprs_2 = CommonExprs::from([ + ( + "test1.c+test1.a".to_string(), + (col(format!("{CSE_PREFIX}_1")), format!("{CSE_PREFIX}_3")), + ), + ( + "test1.b+test1.a".to_string(), + (col(format!("{CSE_PREFIX}_2")), format!("{CSE_PREFIX}_4")), + ), ]); - let project = - build_common_expr_project_plan(join, common_exprs_1, &expr_stats_1).unwrap(); - let project_2 = - build_common_expr_project_plan(project, common_exprs_2, &expr_stats_2) - .unwrap(); + let project = build_common_expr_project_plan(join, common_exprs_1).unwrap(); + let project_2 = build_common_expr_project_plan(project, common_exprs_2).unwrap(); let mut field_set = BTreeSet::new(); for name in project_2.schema().field_names() { @@ -1337,11 +1295,11 @@ mod test { .build()?; let expected = "Projection: test.a, test.b, test.c\ - \n Filter: {Int32(1) + test.a|{test.a}|{Int32(1)}} - Int32(10) > {Int32(1) + test.a|{test.a}|{Int32(1)}}\ - \n Projection: Int32(1) + test.a AS {Int32(1) + test.a|{test.a}|{Int32(1)}}, test.a, test.b, test.c\ + \n Filter: __common_expr_1 - Int32(10) > __common_expr_1\ + \n Projection: Int32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan); + assert_optimized_plan_eq(expected, &plan, None); Ok(()) } @@ -1383,6 +1341,58 @@ mod test { Ok(()) } + #[test] + fn test_alias_collision() -> Result<()> { + let table_scan = test_table_scan()?; + + let config = &OptimizerContext::new(); + let common_expr_1 = config.alias_generator().next(CSE_PREFIX); + let plan = LogicalPlanBuilder::from(table_scan.clone()) + .project(vec![ + (col("a") + col("b")).alias(common_expr_1.clone()), + col("c"), + ])? + .project(vec![ + col(common_expr_1.clone()).alias("c1"), + col(common_expr_1).alias("c2"), + (col("c") + lit(2)).alias("c3"), + (col("c") + lit(2)).alias("c4"), + ])? + .build()?; + + let expected = "Projection: __common_expr_1 AS c1, __common_expr_1 AS c2, __common_expr_2 AS c3, __common_expr_2 AS c4\ + \n Projection: test.c + Int32(2) AS __common_expr_2, __common_expr_1, test.c\ + \n Projection: test.a + test.b AS __common_expr_1, test.c\ + \n TableScan: test"; + + assert_optimized_plan_eq(expected, &plan, Some(config)); + + let config = &OptimizerContext::new(); + let _common_expr_1 = config.alias_generator().next(CSE_PREFIX); + let common_expr_2 = config.alias_generator().next(CSE_PREFIX); + let plan = LogicalPlanBuilder::from(table_scan.clone()) + .project(vec![ + (col("a") + col("b")).alias(common_expr_2.clone()), + col("c"), + ])? + .project(vec![ + col(common_expr_2.clone()).alias("c1"), + col(common_expr_2).alias("c2"), + (col("c") + lit(2)).alias("c3"), + (col("c") + lit(2)).alias("c4"), + ])? + .build()?; + + let expected = "Projection: __common_expr_2 AS c1, __common_expr_2 AS c2, __common_expr_3 AS c3, __common_expr_3 AS c4\ + \n Projection: test.c + Int32(2) AS __common_expr_3, __common_expr_2, test.c\ + \n Projection: test.a + test.b AS __common_expr_2, test.c\ + \n TableScan: test"; + + assert_optimized_plan_eq(expected, &plan, Some(config)); + + Ok(()) + } + #[test] fn test_extract_expressions_from_col() -> Result<()> { let mut result = Vec::with_capacity(1); diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 9e8a2450e0a5..8ccf3ae85345 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4187,8 +4187,8 @@ EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) 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)]] -03)----Aggregate: groupBy=[[t1.y, {CAST(t1.x AS Float64)|{t1.x}} AS t1.x AS alias1]], aggr=[[]] -04)------Projection: CAST(t1.x AS Float64) AS {CAST(t1.x AS Float64)|{t1.x}}, t1.y +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)] @@ -4200,8 +4200,8 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, {CAST(t1.x AS Float64)|{t1.x}}@0 as alias1], aggr=[] -11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as {CAST(t1.x AS Float64)|{t1.x}}, y@1 as y] +10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] +11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] 12)----------------------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index aae35c1ce7bf..c8ef2b7f5e0b 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1436,12 +1436,12 @@ query TT EXPLAIN SELECT x/2, x/2+1 FROM t; ---- logical_plan -01)Projection: {t.x / Int64(2)|{Int64(2)}|{t.x}} AS t.x / Int64(2), {t.x / Int64(2)|{Int64(2)}|{t.x}} AS t.x / Int64(2) + Int64(1) -02)--Projection: t.x / Int64(2) AS {t.x / Int64(2)|{Int64(2)}|{t.x}} +01)Projection: __common_expr_1 AS t.x / Int64(2), __common_expr_1 AS t.x / Int64(2) + Int64(1) +02)--Projection: t.x / Int64(2) AS __common_expr_1 03)----TableScan: t projection=[x] physical_plan -01)ProjectionExec: expr=[{t.x / Int64(2)|{Int64(2)}|{t.x}}@0 as t.x / Int64(2), {t.x / Int64(2)|{Int64(2)}|{t.x}}@0 + 1 as t.x / Int64(2) + Int64(1)] -02)--ProjectionExec: expr=[x@0 / 2 as {t.x / Int64(2)|{Int64(2)}|{t.x}}] +01)ProjectionExec: expr=[__common_expr_1@0 as t.x / Int64(2), __common_expr_1@0 + 1 as t.x / Int64(2) + Int64(1)] +02)--ProjectionExec: expr=[x@0 / 2 as __common_expr_1] 03)----MemoryExec: partitions=1, partition_sizes=[1] query II @@ -1454,12 +1454,12 @@ query TT EXPLAIN SELECT abs(x), abs(x) + abs(y) FROM t; ---- logical_plan -01)Projection: {abs(t.x)|{t.x}} AS abs(t.x), {abs(t.x)|{t.x}} AS abs(t.x) + abs(t.y) -02)--Projection: abs(t.x) AS {abs(t.x)|{t.x}}, t.y +01)Projection: __common_expr_1 AS abs(t.x), __common_expr_1 AS abs(t.x) + abs(t.y) +02)--Projection: abs(t.x) AS __common_expr_1, t.y 03)----TableScan: t projection=[x, y] physical_plan -01)ProjectionExec: expr=[{abs(t.x)|{t.x}}@0 as abs(t.x), {abs(t.x)|{t.x}}@0 + abs(y@1) as abs(t.x) + abs(t.y)] -02)--ProjectionExec: expr=[abs(x@0) as {abs(t.x)|{t.x}}, y@1 as y] +01)ProjectionExec: expr=[__common_expr_1@0 as abs(t.x), __common_expr_1@0 + abs(y@1) as abs(t.x) + abs(t.y)] +02)--ProjectionExec: expr=[abs(x@0) as __common_expr_1, y@1 as y] 03)----MemoryExec: partitions=1, partition_sizes=[1] query II diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index eb0904b2308b..f325d5567603 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1082,8 +1082,8 @@ query TT explain select a/2, a/2 + 1 from t ---- logical_plan -01)Projection: {t.a / Int64(2)|{Int64(2)}|{t.a}} AS t.a / Int64(2), {t.a / Int64(2)|{Int64(2)}|{t.a}} AS t.a / Int64(2) + Int64(1) -02)--Projection: t.a / Int64(2) AS {t.a / Int64(2)|{Int64(2)}|{t.a}} +01)Projection: __common_expr_1 AS t.a / Int64(2), __common_expr_1 AS t.a / Int64(2) + Int64(1) +02)--Projection: t.a / Int64(2) AS __common_expr_1 03)----TableScan: t projection=[a] statement ok @@ -1093,8 +1093,8 @@ query TT explain select a/2, a/2 + 1 from t ---- logical_plan -01)Projection: {t.a / Int64(2)|{Int64(2)}|{t.a}} AS t.a / Int64(2), {t.a / Int64(2)|{Int64(2)}|{t.a}} AS t.a / Int64(2) + Int64(1) -02)--Projection: t.a / Int64(2) AS {t.a / Int64(2)|{Int64(2)}|{t.a}} +01)Projection: __common_expr_1 AS t.a / Int64(2), __common_expr_1 AS t.a / Int64(2) + Int64(1) +02)--Projection: t.a / Int64(2) AS __common_expr_1 03)----TableScan: t projection=[a] ### diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 5e0930b99228..5a21bdf276e3 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -42,8 +42,8 @@ explain select logical_plan 01)Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST 02)--Projection: lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity) AS sum_qty, sum(lineitem.l_extendedprice) AS sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(*) AS count_order -03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum({lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)|{Decimal128(Some(1),20,0) - lineitem.l_discount|{lineitem.l_discount}|{Decimal128(Some(1),20,0)}}|{lineitem.l_extendedprice}}) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum({lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)|{Decimal128(Some(1),20,0) - lineitem.l_discount|{lineitem.l_discount}|{Decimal128(Some(1),20,0)}}|{lineitem.l_extendedprice}} * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(Int64(1)) AS COUNT(*)]] -04)------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS {lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)|{Decimal128(Some(1),20,0) - lineitem.l_discount|{lineitem.l_discount}|{Decimal128(Some(1),20,0)}}|{lineitem.l_extendedprice}}, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus +03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(__common_expr_1) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(Int64(1)) AS COUNT(*)]] +04)------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS __common_expr_1, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus 05)--------Filter: lineitem.l_shipdate <= Date32("1998-09-02") 06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-02")] physical_plan @@ -54,7 +54,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] -08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as {lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)|{Decimal128(Some(1),20,0) - lineitem.l_discount|{lineitem.l_discount}|{Decimal128(Some(1),20,0)}}|{lineitem.l_extendedprice}}, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] +08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: l_shipdate@6 <= 1998-09-02 11)--------------------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_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 99f92b65c3d1..77b839f3f77a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1718,20 +1718,20 @@ EXPLAIN SELECT c3, logical_plan 01)Projection: aggregate_test_100.c3, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 02)--Limit: skip=0, fetch=5 -03)----WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [{aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}} AS aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -04)------Projection: {aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}, aggregate_test_100.c3, aggregate_test_100.c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -05)--------WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [{aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}} AS aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS {aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 +03)----WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [__common_expr_1 AS aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------Projection: __common_expr_1, aggregate_test_100.c3, aggregate_test_100.c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +05)--------WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [__common_expr_1 AS aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS __common_expr_1, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 07)------------TableScan: aggregate_test_100 projection=[c2, c3, c4, c9] physical_plan 01)ProjectionExec: expr=[c3@1 as c3, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2] 02)--GlobalLimitExec: skip=0, fetch=5 03)----WindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 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: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] -04)------ProjectionExec: expr=[{aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}@0 as {aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}, c3@2 as c3, c9@3 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, c3@2 as c3, c9@3 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, 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(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortPreservingMergeExec: [{aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -07)------------SortExec: expr=[{aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST], preserve_partitioning=[true] -08)--------------ProjectionExec: expr=[c3@1 + c4@2 as {aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}, c2@0 as c2, c3@1 as c3, c9@3 as c9] +06)----------SortPreservingMergeExec: [__common_expr_1@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +07)------------SortExec: expr=[__common_expr_1@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true @@ -2574,11 +2574,11 @@ logical_plan 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 -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}} 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: {CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}}, 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({CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}} AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}} AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}} 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({CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}} AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, sum({CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}} AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, sum({CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}} 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 {CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}}, CAST(annotated_data_finite.inc_col AS Int64) AS {CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}}, annotated_data_finite.ts, annotated_data_finite.inc_col, annotated_data_finite.desc_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]] +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] @@ -2586,10 +2586,10 @@ physical_plan 03)----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC], preserve_partitioning=[false] 04)------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] 05)--------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] -06)----------ProjectionExec: expr=[{CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}}@0 as {CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}}, 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)----------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] 07)------------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] 08)--------------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] -09)----------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as {CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}}, CAST(inc_col@1 AS Int64) as {CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}}, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] +09)----------------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] 10)------------------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 query IIIIIIIIIIIIIIIIIIIIIIII @@ -2738,9 +2738,9 @@ logical_plan 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({CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}} 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({CAST(annotated_data_finite.inc_col AS Float64)|{annotated_data_finite.inc_col}} 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({CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}} 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({CAST(annotated_data_finite.inc_col AS Float64)|{annotated_data_finite.inc_col}} 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 {CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}}, CAST(annotated_data_finite.inc_col AS Float64) AS {CAST(annotated_data_finite.inc_col AS Float64)|{annotated_data_finite.inc_col}}, annotated_data_finite.ts, 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] @@ -2749,7 +2749,7 @@ physical_plan 04)------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] 05)--------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] 06)----------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] -07)------------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as {CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}}, CAST(inc_col@1 AS Float64) as {CAST(annotated_data_finite.inc_col AS Float64)|{annotated_data_finite.inc_col}}, ts@0 as ts, inc_col@1 as inc_col] +07)------------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] 08)--------------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 query IIIIIIIIRR @@ -2839,9 +2839,9 @@ logical_plan 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 04)------Projection: sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}} AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}} AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS {CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}}, annotated_data_infinite.ts, annotated_data_infinite.inc_col +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS __common_expr_1, annotated_data_infinite.ts, annotated_data_infinite.inc_col 08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] @@ -2849,7 +2849,7 @@ physical_plan 03)----ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as {CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}}, ts@0 as ts, inc_col@1 as inc_col] +06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] 07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2886,9 +2886,9 @@ logical_plan 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 04)------Projection: sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}} AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}} AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS {CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}}, annotated_data_infinite.ts, annotated_data_infinite.inc_col +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS __common_expr_1, annotated_data_infinite.ts, annotated_data_infinite.inc_col 08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] @@ -2896,7 +2896,7 @@ physical_plan 03)----ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as {CAST(annotated_data_infinite.inc_col AS Int64)|{annotated_data_infinite.inc_col}}, ts@0 as ts, inc_col@1 as inc_col] +06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] 07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] @@ -2983,13 +2983,13 @@ EXPLAIN SELECT a, b, c, logical_plan 01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 02)--Limit: skip=0, fetch=5 -03)----WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] -04)------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] -07)------------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] -08)--------------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}} AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -09)----------------Projection: CAST(annotated_data_infinite2.c AS Int64) AS {CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}}, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +03)----WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] +04)------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] +07)------------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] +08)--------------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +09)----------------Projection: CAST(annotated_data_infinite2.c AS Int64) AS __common_expr_1, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d 10)------------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan 01)ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] @@ -3000,7 +3000,7 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] 07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] 08)--------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 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(5)), is_causal: false }], mode=[Sorted] -09)----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as {CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}}, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +09)----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 10)------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query IIIIIIIIIIIIIII @@ -3052,13 +3052,13 @@ logical_plan 01)Limit: skip=0, fetch=5 02)--Sort: annotated_data_finite2.c ASC NULLS LAST, fetch=5 03)----Projection: annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 -04)------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -07)------------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] -08)--------------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] -09)----------------WindowAggr: windowExpr=[[sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum({CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}} AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -10)------------------Projection: CAST(annotated_data_finite2.c AS Int64) AS {CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}}, annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, annotated_data_finite2.d +04)------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +07)------------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] +09)----------------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, sum(__common_expr_1 AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +10)------------------Projection: CAST(annotated_data_finite2.c AS Int64) AS __common_expr_1, annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, annotated_data_finite2.d 11)--------------------TableScan: annotated_data_finite2 projection=[a, b, c, d] physical_plan 01)GlobalLimitExec: skip=0, fetch=5 @@ -3075,7 +3075,7 @@ physical_plan 12)----------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] 13)------------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] 14)--------------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 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(5)), is_causal: false }], mode=[Sorted] -15)----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as {CAST(annotated_data_finite2.c AS Int64)|{annotated_data_finite2.c}}, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +15)----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIII @@ -3241,21 +3241,21 @@ FROM annotated_data_infinite2; ---- logical_plan 01)Projection: sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 -02)--WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -03)----Projection: {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, annotated_data_infinite2.a, annotated_data_infinite2.d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -04)------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -06)----------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +02)--WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: __common_expr_1, annotated_data_infinite2.a, annotated_data_infinite2.d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS __common_expr_1, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d 08)--------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan 01)ProjectionExec: expr=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] 02)--BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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=[Linear] -03)----ProjectionExec: expr=[{CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}@0 as {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, a@1 as a, d@4 as d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +03)----ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, a@1 as a, d@4 as d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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=[PartiallySorted([0])] 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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] -07)------------ProjectionExec: expr=[CAST(a@0 AS Int64) as {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +07)------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 08)--------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] statement ok @@ -3272,29 +3272,29 @@ FROM annotated_data_infinite2; ---- logical_plan 01)Projection: sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 -02)--WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -03)----Projection: {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, annotated_data_infinite2.a, annotated_data_infinite2.d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -04)------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -05)--------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -06)----------WindowAggr: windowExpr=[[sum({CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}} AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +02)--WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: __common_expr_1, annotated_data_infinite2.a, annotated_data_infinite2.d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS __common_expr_1, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d 08)--------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan 01)ProjectionExec: expr=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] 02)--BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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=[Linear] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs={CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}@0 ASC NULLS LAST,a@1 ASC NULLS LAST -05)--------ProjectionExec: expr=[{CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}@0 as {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, a@1 as a, d@4 as d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=__common_expr_1@0 ASC NULLS LAST,a@1 ASC NULLS LAST +05)--------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, a@1 as a, d@4 as d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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] 07)------------CoalesceBatchesExec: target_batch_size=4096 -08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,{CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}@0 ASC NULLS LAST +08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST 09)----------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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=[PartiallySorted([0])] 10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,{CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}@0 ASC NULLS LAST +11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST 12)----------------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, 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] 13)------------------------CoalesceBatchesExec: target_batch_size=4096 -14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,{CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}@0 ASC NULLS LAST -15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as {CAST(annotated_data_infinite2.a AS Int64)|{annotated_data_infinite2.a}}, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] From b26c1b819dff7ed1b48ab20c66b4bd1226ff8d79 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 18 Jun 2024 04:02:59 -0700 Subject: [PATCH 13/54] fix: Fix the incorrect null joined rows for SMJ outer join with join filter (#10892) * fix: Fix the incorrect null joined rows for outer join with join filter * Update datafusion/physical-plan/src/joins/sort_merge_join.rs Co-authored-by: Oleks V * Update datafusion/physical-plan/src/joins/sort_merge_join.rs Co-authored-by: Oleks V * Update datafusion/physical-plan/src/joins/sort_merge_join.rs Co-authored-by: Oleks V * Update datafusion/physical-plan/src/joins/sort_merge_join.rs Co-authored-by: Oleks V * For review --------- Co-authored-by: Oleks V --- .../src/joins/sort_merge_join.rs | 278 +++++++++++------- .../test_files/sort_merge_join.slt | 29 +- 2 files changed, 194 insertions(+), 113 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 01abb30181d0..420fab51da39 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -46,6 +46,7 @@ use arrow::array::*; use arrow::compute::{self, concat_batches, take, SortOptions}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::ArrowError; +use arrow_array::types::UInt64Type; use datafusion_common::{ internal_err, not_impl_err, plan_err, DataFusionError, JoinSide, JoinType, Result, @@ -471,6 +472,7 @@ struct StreamedJoinedChunk { /// Array builder for streamed indices streamed_indices: UInt64Builder, /// Array builder for buffered indices + /// This could contain nulls if the join is null-joined buffered_indices: UInt64Builder, } @@ -559,6 +561,9 @@ struct BufferedBatch { pub null_joined: Vec, /// Size estimation used for reserving / releasing memory pub size_estimation: usize, + /// The indices of buffered batch that failed the join filter. + /// When dequeuing the buffered batch, we need to produce null joined rows for these indices. + pub join_filter_failed_idxs: HashSet, } impl BufferedBatch { @@ -590,6 +595,7 @@ impl BufferedBatch { join_arrays, null_joined: vec![], size_estimation, + join_filter_failed_idxs: HashSet::new(), } } } @@ -847,6 +853,7 @@ impl SMJStream { // pop previous buffered batches while !self.buffered_data.batches.is_empty() { let head_batch = self.buffered_data.head_batch(); + // If the head batch is fully processed, dequeue it and produce output of it. if head_batch.range.end == head_batch.batch.num_rows() { self.freeze_dequeuing_buffered()?; if let Some(buffered_batch) = @@ -855,6 +862,8 @@ impl SMJStream { self.reservation.shrink(buffered_batch.size_estimation); } } else { + // If the head batch is not fully processed, break the loop. + // Streamed batch will be joined with the head batch in the next step. break; } } @@ -1050,7 +1059,7 @@ impl SMJStream { Some(scanning_idx), ); } else { - // Join nulls and buffered row + // Join nulls and buffered row for FULL join self.buffered_data .scanning_batch_mut() .null_joined @@ -1083,7 +1092,7 @@ impl SMJStream { fn freeze_all(&mut self) -> Result<()> { self.freeze_streamed()?; - self.freeze_buffered(self.buffered_data.batches.len())?; + self.freeze_buffered(self.buffered_data.batches.len(), false)?; Ok(()) } @@ -1093,7 +1102,8 @@ impl SMJStream { // 2. freezes NULLs joined to dequeued buffered batch to "release" it fn freeze_dequeuing_buffered(&mut self) -> Result<()> { self.freeze_streamed()?; - self.freeze_buffered(1)?; + // Only freeze and produce the first batch in buffered_data as the batch is fully processed + self.freeze_buffered(1, true)?; Ok(()) } @@ -1101,7 +1111,14 @@ impl SMJStream { // NULLs on streamed side. // // Applicable only in case of Full join. - fn freeze_buffered(&mut self, batch_count: usize) -> Result<()> { + // + // If `output_not_matched_filter` is true, this will also produce record batches + // for buffered rows which are joined with streamed side but don't match join filter. + fn freeze_buffered( + &mut self, + batch_count: usize, + output_not_matched_filter: bool, + ) -> Result<()> { if !matches!(self.join_type, JoinType::Full) { return Ok(()); } @@ -1109,33 +1126,31 @@ impl SMJStream { let buffered_indices = UInt64Array::from_iter_values( buffered_batch.null_joined.iter().map(|&index| index as u64), ); - if buffered_indices.is_empty() { - continue; + if let Some(record_batch) = produce_buffered_null_batch( + &self.schema, + &self.streamed_schema, + &buffered_indices, + buffered_batch, + )? { + self.output_record_batches.push(record_batch); } buffered_batch.null_joined.clear(); - // Take buffered (right) columns - let buffered_columns = buffered_batch - .batch - .columns() - .iter() - .map(|column| take(column, &buffered_indices, None)) - .collect::, ArrowError>>() - .map_err(Into::::into)?; - - // Create null streamed (left) columns - let mut streamed_columns = self - .streamed_schema - .fields() - .iter() - .map(|f| new_null_array(f.data_type(), buffered_indices.len())) - .collect::>(); - - streamed_columns.extend(buffered_columns); - let columns = streamed_columns; - - self.output_record_batches - .push(RecordBatch::try_new(self.schema.clone(), columns)?); + // For buffered rows which are joined with streamed side but doesn't satisfy the join filter + if output_not_matched_filter { + let buffered_indices = UInt64Array::from_iter_values( + buffered_batch.join_filter_failed_idxs.iter().copied(), + ); + if let Some(record_batch) = produce_buffered_null_batch( + &self.schema, + &self.streamed_schema, + &buffered_indices, + buffered_batch, + )? { + self.output_record_batches.push(record_batch); + } + buffered_batch.join_filter_failed_idxs.clear(); + } } Ok(()) } @@ -1144,6 +1159,7 @@ impl SMJStream { // for current streamed batch and clears staged output indices. fn freeze_streamed(&mut self) -> Result<()> { for chunk in self.streamed_batch.output_indices.iter_mut() { + // The row indices of joined streamed batch let streamed_indices = chunk.streamed_indices.finish(); if streamed_indices.is_empty() { @@ -1158,6 +1174,7 @@ impl SMJStream { .map(|column| take(column, &streamed_indices, None)) .collect::, ArrowError>>()?; + // The row indices of joined buffered batch let buffered_indices: UInt64Array = chunk.buffered_indices.finish(); let mut buffered_columns = if matches!(self.join_type, JoinType::LeftSemi | JoinType::LeftAnti) { @@ -1169,6 +1186,8 @@ impl SMJStream { &buffered_indices, )? } else { + // If buffered batch none, meaning it is null joined batch. + // We need to create null arrays for buffered columns to join with streamed rows. self.buffered_schema .fields() .iter() @@ -1200,7 +1219,8 @@ impl SMJStream { get_filter_column(&self.filter, &streamed_columns, &buffered_columns) } } else { - // This chunk is for null joined rows (outer join), we don't need to apply join filter. + // This chunk is totally for null joined rows (outer join), we don't need to apply join filter. + // Any join filter applied only on either streamed or buffered side will be pushed already. vec![] }; @@ -1229,49 +1249,73 @@ impl SMJStream { .evaluate(&filter_batch)? .into_array(filter_batch.num_rows())?; - // The selection mask of the filter - let mut mask = + // The boolean selection mask of the join filter result + let pre_mask = datafusion_common::cast::as_boolean_array(&filter_result)?; + // If there are nulls in join filter result, exclude them from selecting + // the rows to output. + let mask = if pre_mask.null_count() > 0 { + compute::prep_null_mask_filter( + datafusion_common::cast::as_boolean_array(&filter_result)?, + ) + } else { + pre_mask.clone() + }; + + // For certain join types, we need to adjust the initial mask to handle the join filter. let maybe_filtered_join_mask: Option<(BooleanArray, Vec)> = get_filtered_join_mask( self.join_type, - streamed_indices, - mask, + &streamed_indices, + &mask, &self.streamed_batch.join_filter_matched_idxs, &self.buffered_data.scanning_offset, ); - if let Some(ref filtered_join_mask) = maybe_filtered_join_mask { - mask = &filtered_join_mask.0; - self.streamed_batch - .join_filter_matched_idxs - .extend(&filtered_join_mask.1); - } + let mask = + if let Some(ref filtered_join_mask) = maybe_filtered_join_mask { + self.streamed_batch + .join_filter_matched_idxs + .extend(&filtered_join_mask.1); + &filtered_join_mask.0 + } else { + &mask + }; - // Push the filtered batch to the output + // Push the filtered batch which contains rows passing join filter to the output let filtered_batch = compute::filter_record_batch(&output_batch, mask)?; self.output_record_batches.push(filtered_batch); - // For outer joins, we need to push the null joined rows to the output. + // For outer joins, we need to push the null joined rows to the output if + // all joined rows are failed on the join filter. + // I.e., if all rows joined from a streamed row are failed with the join filter, + // we need to join it with nulls as buffered side. if matches!( self.join_type, JoinType::Left | JoinType::Right | JoinType::Full ) { - // The reverse of the selection mask. For the rows not pass join filter above, - // we need to join them (left or right) with null rows for outer joins. - let not_mask = if mask.null_count() > 0 { - // If the mask contains nulls, we need to use `prep_null_mask_filter` to - // handle the nulls in the mask as false to produce rows where the mask - // was null itself. - compute::not(&compute::prep_null_mask_filter(mask))? - } else { - compute::not(mask)? - }; + // We need to get the mask for row indices that the joined rows are failed + // on the join filter. I.e., for a row in streamed side, if all joined rows + // between it and all buffered rows are failed on the join filter, we need to + // output it with null columns from buffered side. For the mask here, it + // behaves like LeftAnti join. + let null_mask: BooleanArray = get_filtered_join_mask( + // Set a mask slot as true only if all joined rows of same streamed index + // are failed on the join filter. + // The masking behavior is like LeftAnti join. + JoinType::LeftAnti, + &streamed_indices, + mask, + &self.streamed_batch.join_filter_matched_idxs, + &self.buffered_data.scanning_offset, + ) + .unwrap() + .0; let null_joined_batch = - compute::filter_record_batch(&output_batch, ¬_mask)?; + compute::filter_record_batch(&output_batch, &null_mask)?; let mut buffered_columns = self .buffered_schema @@ -1308,51 +1352,37 @@ impl SMJStream { streamed_columns }; + // Push the streamed/buffered batch joined nulls to the output let null_joined_streamed_batch = RecordBatch::try_new(self.schema.clone(), columns.clone())?; self.output_record_batches.push(null_joined_streamed_batch); - // For full join, we also need to output the null joined rows from the buffered side + // For full join, we also need to output the null joined rows from the buffered side. + // Usually this is done by `freeze_buffered`. However, if a buffered row is joined with + // streamed side, it won't be outputted by `freeze_buffered`. + // We need to check if a buffered row is joined with streamed side and output. + // If it is joined with streamed side, but doesn't match the join filter, + // we need to output it with nulls as streamed side. if matches!(self.join_type, JoinType::Full) { - // Handle not mask for buffered side further. - // For buffered side, we want to output the rows that are not null joined with - // the streamed side. i.e. the rows that are not null in the `buffered_indices`. - let not_mask = if let Some(nulls) = buffered_indices.nulls() { - let mask = not_mask.values() & nulls.inner(); - BooleanArray::new(mask, None) - } else { - not_mask - }; - - let null_joined_batch = - compute::filter_record_batch(&output_batch, ¬_mask)?; - - let mut streamed_columns = self - .streamed_schema - .fields() - .iter() - .map(|f| { - new_null_array( - f.data_type(), - null_joined_batch.num_rows(), - ) - }) - .collect::>(); - - let buffered_columns = null_joined_batch - .columns() - .iter() - .skip(streamed_columns_length) - .cloned() - .collect::>(); - - streamed_columns.extend(buffered_columns); - - let null_joined_buffered_batch = RecordBatch::try_new( - self.schema.clone(), - streamed_columns, - )?; - self.output_record_batches.push(null_joined_buffered_batch); + for i in 0..pre_mask.len() { + let buffered_batch = &mut self.buffered_data.batches + [chunk.buffered_batch_idx.unwrap()]; + let buffered_index = buffered_indices.value(i); + + if !pre_mask.value(i) { + // For a buffered row that is joined with streamed side but doesn't satisfy the join filter, + buffered_batch + .join_filter_failed_idxs + .insert(buffered_index); + } else if buffered_batch + .join_filter_failed_idxs + .contains(&buffered_index) + { + buffered_batch + .join_filter_failed_idxs + .remove(&buffered_index); + } + } } } } else { @@ -1417,6 +1447,40 @@ fn get_filter_column( filter_columns } +fn produce_buffered_null_batch( + schema: &SchemaRef, + streamed_schema: &SchemaRef, + buffered_indices: &PrimitiveArray, + buffered_batch: &BufferedBatch, +) -> Result> { + if buffered_indices.is_empty() { + return Ok(None); + } + + // Take buffered (right) columns + let buffered_columns = buffered_batch + .batch + .columns() + .iter() + .map(|column| take(column, &buffered_indices, None)) + .collect::, ArrowError>>() + .map_err(Into::::into)?; + + // Create null streamed (left) columns + let mut streamed_columns = streamed_schema + .fields() + .iter() + .map(|f| new_null_array(f.data_type(), buffered_indices.len())) + .collect::>(); + + streamed_columns.extend(buffered_columns); + + Ok(Some(RecordBatch::try_new( + schema.clone(), + streamed_columns, + )?)) +} + /// Get `buffered_indices` rows for `buffered_data[buffered_batch_idx]` #[inline(always)] fn get_buffered_columns( @@ -1440,9 +1504,13 @@ fn get_buffered_columns( /// `streamed_indices` have the same length as `mask` /// `matched_indices` array of streaming indices that already has a join filter match /// `scanning_buffered_offset` current buffered offset across batches +/// +/// This return a tuple of: +/// - corrected mask with respect to the join type +/// - indices of rows in streamed batch that have a join filter match fn get_filtered_join_mask( join_type: JoinType, - streamed_indices: UInt64Array, + streamed_indices: &UInt64Array, mask: &BooleanArray, matched_indices: &HashSet, scanning_buffered_offset: &usize, @@ -2803,7 +2871,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftSemi, - UInt64Array::from(vec![0, 0, 1, 1]), + &UInt64Array::from(vec![0, 0, 1, 1]), &BooleanArray::from(vec![true, true, false, false]), &HashSet::new(), &0, @@ -2814,7 +2882,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftSemi, - UInt64Array::from(vec![0, 1]), + &UInt64Array::from(vec![0, 1]), &BooleanArray::from(vec![true, true]), &HashSet::new(), &0, @@ -2825,7 +2893,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftSemi, - UInt64Array::from(vec![0, 1]), + &UInt64Array::from(vec![0, 1]), &BooleanArray::from(vec![false, true]), &HashSet::new(), &0, @@ -2836,7 +2904,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftSemi, - UInt64Array::from(vec![0, 1]), + &UInt64Array::from(vec![0, 1]), &BooleanArray::from(vec![true, false]), &HashSet::new(), &0, @@ -2847,7 +2915,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftSemi, - UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), + &UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), &BooleanArray::from(vec![false, true, true, true, true, true]), &HashSet::new(), &0, @@ -2861,7 +2929,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftSemi, - UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), + &UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), &BooleanArray::from(vec![false, false, false, false, false, true]), &HashSet::new(), &0, @@ -2880,7 +2948,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftAnti, - UInt64Array::from(vec![0, 0, 1, 1]), + &UInt64Array::from(vec![0, 0, 1, 1]), &BooleanArray::from(vec![true, true, false, false]), &HashSet::new(), &0, @@ -2891,7 +2959,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftAnti, - UInt64Array::from(vec![0, 1]), + &UInt64Array::from(vec![0, 1]), &BooleanArray::from(vec![true, true]), &HashSet::new(), &0, @@ -2902,7 +2970,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftAnti, - UInt64Array::from(vec![0, 1]), + &UInt64Array::from(vec![0, 1]), &BooleanArray::from(vec![false, true]), &HashSet::new(), &0, @@ -2913,7 +2981,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftAnti, - UInt64Array::from(vec![0, 1]), + &UInt64Array::from(vec![0, 1]), &BooleanArray::from(vec![true, false]), &HashSet::new(), &0, @@ -2924,7 +2992,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftAnti, - UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), + &UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), &BooleanArray::from(vec![false, true, true, true, true, true]), &HashSet::new(), &0, @@ -2938,7 +3006,7 @@ mod tests { assert_eq!( get_filtered_join_mask( LeftAnti, - UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), + &UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), &BooleanArray::from(vec![false, false, false, false, false, true]), &HashSet::new(), &0, diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index b4deb43a728e..5a6334602c22 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -84,7 +84,6 @@ SELECT * FROM t1 LEFT JOIN t2 ON t1.a = t2.a AND t2.b * 50 <= t1.b Alice 100 Alice 1 Alice 100 Alice 2 Alice 50 Alice 1 -Alice 50 NULL NULL Bob 1 NULL NULL query TITI rowsort @@ -112,7 +111,6 @@ SELECT * FROM t1 RIGHT JOIN t2 ON t1.a = t2.a AND t2.b * 50 <= t1.b Alice 100 Alice 1 Alice 100 Alice 2 Alice 50 Alice 1 -NULL NULL Alice 2 query TITI rowsort SELECT * FROM t1 RIGHT JOIN t2 ON t1.a = t2.a AND t1.b > t2.b @@ -137,12 +135,9 @@ query TITI rowsort SELECT * FROM t1 FULL JOIN t2 ON t1.a = t2.a AND t2.b * 50 > t1.b ---- Alice 100 NULL NULL -Alice 100 NULL NULL Alice 50 Alice 2 -Alice 50 NULL NULL Bob 1 NULL NULL NULL NULL Alice 1 -NULL NULL Alice 1 NULL NULL Alice 2 query TITI rowsort @@ -151,10 +146,7 @@ SELECT * FROM t1 FULL JOIN t2 ON t1.a = t2.a AND t1.b > t2.b + 50 Alice 100 Alice 1 Alice 100 Alice 2 Alice 50 NULL NULL -Alice 50 NULL NULL Bob 1 NULL NULL -NULL NULL Alice 1 -NULL NULL Alice 2 statement ok DROP TABLE t1; @@ -613,6 +605,27 @@ select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1. ) order by 1, 2 ---- +query IIII +select * from ( +with t as ( + select id, id % 5 id1 from (select unnest(range(0,10)) id) +), t1 as ( + select id % 10 id, id + 2 id1 from (select unnest(range(0,10)) id) +) +select * from t right join t1 on t.id1 = t1.id and t.id > t1.id1 +) order by 1, 2, 3, 4 +---- +5 0 0 2 +6 1 1 3 +7 2 2 4 +8 3 3 5 +9 4 4 6 +NULL NULL 5 7 +NULL NULL 6 8 +NULL NULL 7 9 +NULL NULL 8 10 +NULL NULL 9 11 + # return sql params back to default values statement ok set datafusion.optimizer.prefer_hash_join = true; From 500b73f996d871e4f7c1ec508413f8a2e312756f Mon Sep 17 00:00:00 2001 From: Xin Li <33629085+xinlifoobar@users.noreply.github.com> Date: Tue, 18 Jun 2024 21:27:15 +0800 Subject: [PATCH 14/54] Substrait support for ParquetExec round trip for simple select (#10949) * physical plan support no statistics * implement simple physical plan substrait * fix ci * fix misc * Add README.md for substrait test data --- .../substrait/src/physical_plan/consumer.rs | 87 ++++++++++++++-- .../substrait/src/physical_plan/producer.rs | 93 +++++++++++++++++- .../tests/cases/roundtrip_physical_plan.rs | 92 ++++++++++++++++- datafusion/substrait/tests/testdata/Readme.md | 51 ++++++++++ .../substrait/tests/testdata/data.parquet | Bin 0 -> 4342 bytes .../substrait/tests/testdata/empty.parquet | Bin 0 -> 976 bytes 6 files changed, 308 insertions(+), 15 deletions(-) create mode 100644 datafusion/substrait/tests/testdata/Readme.md create mode 100644 datafusion/substrait/tests/testdata/data.parquet create mode 100644 datafusion/substrait/tests/testdata/empty.parquet diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 39b38c94ec18..5a8b888ef1cc 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -18,8 +18,8 @@ use std::collections::HashMap; use std::sync::Arc; -use datafusion::arrow::datatypes::Schema; -use datafusion::common::not_impl_err; +use datafusion::arrow::datatypes::{DataType, Field, Schema}; +use datafusion::common::{not_impl_err, substrait_err}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; @@ -30,7 +30,9 @@ use datafusion::prelude::SessionContext; use async_recursion::async_recursion; use chrono::DateTime; use object_store::ObjectMeta; +use substrait::proto::r#type::{Kind, Nullability}; use substrait::proto::read_rel::local_files::file_or_files::PathType; +use substrait::proto::Type; use substrait::proto::{ expression::MaskExpression, read_rel::ReadType, rel::RelType, Rel, }; @@ -42,17 +44,42 @@ pub async fn from_substrait_rel( rel: &Rel, _extensions: &HashMap, ) -> Result> { + let mut base_config; + match &rel.rel_type { Some(RelType::Read(read)) => { if read.filter.is_some() || read.best_effort_filter.is_some() { return not_impl_err!("Read with filter is not supported"); } - if read.base_schema.is_some() { - return not_impl_err!("Read with schema is not supported"); - } + if read.advanced_extension.is_some() { return not_impl_err!("Read with AdvancedExtension is not supported"); } + + let Some(schema) = read.base_schema.as_ref() else { + return substrait_err!("Missing base schema in the read"); + }; + + let Some(r#struct) = schema.r#struct.as_ref() else { + return substrait_err!("Missing struct in the schema"); + }; + + match schema + .names + .iter() + .zip(r#struct.types.iter()) + .map(|(name, r#type)| to_field(name, r#type)) + .collect::>>() + { + Ok(fields) => { + base_config = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + Arc::new(Schema::new(fields)), + ); + } + Err(e) => return Err(e), + }; + match &read.as_ref().read_type { Some(ReadType::LocalFiles(files)) => { let mut file_groups = vec![]; @@ -104,11 +131,7 @@ pub async fn from_substrait_rel( file_groups[part_index].push(partitioned_file) } - let mut base_config = FileScanConfig::new( - ObjectStoreUrl::local_filesystem(), - Arc::new(Schema::empty()), - ) - .with_file_groups(file_groups); + base_config = base_config.with_file_groups(file_groups); if let Some(MaskExpression { select, .. }) = &read.projection { if let Some(projection) = &select.as_ref() { @@ -132,3 +155,47 @@ pub async fn from_substrait_rel( _ => not_impl_err!("Unsupported RelType: {:?}", rel.rel_type), } } + +fn to_field(name: &String, r#type: &Type) -> Result { + let Some(kind) = r#type.kind.as_ref() else { + return substrait_err!("Missing kind in the type with name {}", name); + }; + + let mut nullable = false; + let data_type = match kind { + Kind::Bool(boolean) => { + nullable = is_nullable(boolean.nullability); + Ok(DataType::Boolean) + } + Kind::I64(i64) => { + nullable = is_nullable(i64.nullability); + Ok(DataType::Int64) + } + Kind::Fp64(fp64) => { + nullable = is_nullable(fp64.nullability); + Ok(DataType::Float64) + } + Kind::String(string) => { + nullable = is_nullable(string.nullability); + Ok(DataType::Utf8) + } + _ => substrait_err!( + "Unsupported kind: {:?} in the type with name {}", + kind, + name + ), + }?; + + Ok(Field::new(name, data_type, nullable)) +} + +fn is_nullable(nullability: i32) -> bool { + let Ok(nullability) = Nullability::try_from(nullability) else { + return true; + }; + + match nullability { + Nullability::Nullable | Nullability::Unspecified => true, + Nullability::Required => false, + } +} diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index ad87d7afb058..57fe68c4a780 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -15,12 +15,16 @@ // specific language governing permissions and limitations // under the License. +use datafusion::arrow::datatypes::DataType; use datafusion::datasource::physical_plan::ParquetExec; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::{displayable, ExecutionPlan}; use std::collections::HashMap; +use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; use substrait::proto::expression::MaskExpression; -use substrait::proto::extensions; +use substrait::proto::r#type::{ + Boolean, Fp64, Kind, Nullability, String as SubstraitString, Struct, I64, +}; use substrait::proto::read_rel::local_files::file_or_files::ParquetReadOptions; use substrait::proto::read_rel::local_files::file_or_files::{FileFormat, PathType}; use substrait::proto::read_rel::local_files::FileOrFiles; @@ -29,6 +33,7 @@ use substrait::proto::read_rel::ReadType; use substrait::proto::rel::RelType; use substrait::proto::ReadRel; use substrait::proto::Rel; +use substrait::proto::{extensions, NamedStruct, Type}; /// Convert DataFusion ExecutionPlan to Substrait Rel pub fn to_substrait_rel( @@ -55,15 +60,56 @@ pub fn to_substrait_rel( } } + let mut names = vec![]; + let mut types = vec![]; + + for field in base_config.file_schema.fields.iter() { + match to_substrait_type(field.data_type(), field.is_nullable()) { + Ok(t) => { + names.push(field.name().clone()); + types.push(t); + } + Err(e) => return Err(e), + } + } + + let type_info = Struct { + types, + // FIXME: duckdb doesn't set this field, keep it as default variant 0. + // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1106-L1127 + type_variation_reference: 0, + nullability: Nullability::Required.into(), + }; + + let mut select_struct = None; + if let Some(projection) = base_config.projection.as_ref() { + let struct_items = projection + .iter() + .map(|index| StructItem { + field: *index as i32, + // FIXME: duckdb sets this to None, but it's not clear why. + // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1191 + child: None, + }) + .collect(); + + select_struct = Some(StructSelect { struct_items }); + } + Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { common: None, - base_schema: None, + base_schema: Some(NamedStruct { + names, + r#struct: Some(type_info), + }), filter: None, best_effort_filter: None, projection: Some(MaskExpression { - select: None, - maintain_singular_struct: false, + select: select_struct, + // FIXME: duckdb set this to true, but it's not clear why. + // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1186. + maintain_singular_struct: true, }), advanced_extension: None, read_type: Some(ReadType::LocalFiles(LocalFiles { @@ -79,3 +125,42 @@ pub fn to_substrait_rel( ))) } } + +// see https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L954-L1094. +fn to_substrait_type(data_type: &DataType, nullable: bool) -> Result { + let nullability = if nullable { + Nullability::Nullable.into() + } else { + Nullability::Required.into() + }; + + match data_type { + DataType::Boolean => Ok(Type { + kind: Some(Kind::Bool(Boolean { + type_variation_reference: 0, + nullability, + })), + }), + DataType::Int64 => Ok(Type { + kind: Some(Kind::I64(I64 { + type_variation_reference: 0, + nullability, + })), + }), + DataType::Float64 => Ok(Type { + kind: Some(Kind::Fp64(Fp64 { + type_variation_reference: 0, + nullability, + })), + }), + DataType::Utf8 => Ok(Type { + kind: Some(Kind::String(SubstraitString { + type_variation_reference: 0, + nullability, + })), + }), + _ => Err(DataFusionError::Substrait(format!( + "Logical type {data_type} not implemented as substrait type" + ))), + } +} diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 4014670a7cbc..57fb3e2ee7cc 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -19,12 +19,13 @@ use std::collections::HashMap; use std::sync::Arc; use datafusion::arrow::datatypes::Schema; +use datafusion::dataframe::DataFrame; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::error::Result; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use datafusion::prelude::SessionContext; +use datafusion::prelude::{ParquetReadOptions, SessionContext}; use datafusion_substrait::physical_plan::{consumer, producer}; use substrait::proto::extensions; @@ -71,3 +72,92 @@ async fn parquet_exec() -> Result<()> { Ok(()) } + +#[tokio::test] +async fn simple_select() -> Result<()> { + roundtrip("SELECT a, b FROM data").await +} + +#[tokio::test] +#[ignore = "This test is failing because the translation of the substrait plan to the physical plan is not implemented yet"] +async fn simple_select_alltypes() -> Result<()> { + roundtrip_alltypes("SELECT bool_col, int_col FROM alltypes_plain").await +} + +#[tokio::test] +async fn wildcard_select() -> Result<()> { + roundtrip("SELECT * FROM data").await +} + +#[tokio::test] +#[ignore = "This test is failing because the translation of the substrait plan to the physical plan is not implemented yet"] +async fn wildcard_select_alltypes() -> Result<()> { + roundtrip_alltypes("SELECT * FROM alltypes_plain").await +} + +async fn roundtrip(sql: &str) -> Result<()> { + let ctx = create_parquet_context().await?; + let df = ctx.sql(sql).await?; + + roundtrip_parquet(df).await?; + + Ok(()) +} + +async fn roundtrip_alltypes(sql: &str) -> Result<()> { + let ctx = create_all_types_context().await?; + let df = ctx.sql(sql).await?; + + roundtrip_parquet(df).await?; + + Ok(()) +} + +async fn roundtrip_parquet(df: DataFrame) -> Result<()> { + let physical_plan = df.create_physical_plan().await?; + + // Convert the plan into a substrait (protobuf) Rel + let mut extension_info = (vec![], HashMap::new()); + let substrait_plan = + producer::to_substrait_rel(physical_plan.as_ref(), &mut extension_info)?; + + // Convert the substrait Rel back into a physical plan + let ctx = create_parquet_context().await?; + let physical_plan_roundtrip = + consumer::from_substrait_rel(&ctx, substrait_plan.as_ref(), &HashMap::new()) + .await?; + + // Compare the original and roundtrip physical plans + let expected = format!("{}", displayable(physical_plan.as_ref()).indent(true)); + let actual = format!( + "{}", + displayable(physical_plan_roundtrip.as_ref()).indent(true) + ); + assert_eq!(expected, actual); + + Ok(()) +} + +async fn create_parquet_context() -> Result { + let ctx = SessionContext::new(); + let explicit_options = ParquetReadOptions::default(); + + ctx.register_parquet("data", "tests/testdata/data.parquet", explicit_options) + .await?; + + Ok(ctx) +} + +async fn create_all_types_context() -> Result { + let ctx = SessionContext::new(); + + let testdata = datafusion::test_util::parquet_test_data(); + ctx.register_parquet( + "alltypes_plain", + &format!("{testdata}/alltypes_plain.parquet"), + ParquetReadOptions::default(), + ) + .await?; + + Ok(ctx) +} diff --git a/datafusion/substrait/tests/testdata/Readme.md b/datafusion/substrait/tests/testdata/Readme.md new file mode 100644 index 000000000000..c1bd48abf96e --- /dev/null +++ b/datafusion/substrait/tests/testdata/Readme.md @@ -0,0 +1,51 @@ + + +# Apache DataFusion Substrait Test Data + +This folder contains test data for the [substrait] crate. + +The substrait crate is at an init stage and many functions not implemented yet. Compared to the [parquet-testing](https://github.com/apache/parquet-testing) submodule, this folder contains only simple test data evolving around the substrait producers and consumers for [logical plans](https://github.com/apache/datafusion/tree/main/datafusion/substrait/src/logical_plan) and [physical plans](https://github.com/apache/datafusion/tree/main/datafusion/substrait/src/physical_plan). + +## Test Data + +### Example Data + +- [empty.csv](https://github.com/apache/datafusion/blob/main/datafusion/substrait/tests/testdata/empty.csv): An empty CSV file. +- [empty.parquet](https://github.com/apache/datafusion/blob/main/datafusion/substrait/tests/testdata/empty.parquet): An empty Parquet file with metadata only. +- [data.csv](https://github.com/apache/datafusion/blob/main/datafusion/substrait/tests/testdata/data.csv): A simple CSV file with 6 columns and 2 rows. +- [data.parquet](https://github.com/apache/datafusion/blob/main/datafusion/substrait/tests/testdata/data.parquet): A simple Parquet generated from the CSV file using `pandas`, e.g., + + ```python + import pandas as pd + + df = pandas.read_csv('data.csv') + df.to_parquet('data.parquet') + ``` + +### Add new test data + +To add a new test data, create a new file in this folder, reference it in the test source file, e.g., + +```rust +let ctx = SessionContext::new(); +let explicit_options = ParquetReadOptions::default(); + +ctx.register_parquet("data", "tests/testdata/data.parquet", explicit_options) +``` diff --git a/datafusion/substrait/tests/testdata/data.parquet b/datafusion/substrait/tests/testdata/data.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f9c03394db436c39dc10a6f7a1dcd302001b4716 GIT binary patch literal 4342 zcmcIo&2QpH6gLSX*))lws76NOfKb=jL$V7Fq?^@NT0;UcP1wLD7^teqV;eBB4a;ZP zZKOT+RP|U@>TQobRs9os=&9;q4^lC&2*hg8#HO7yRAF97D0+^9*BT5I<#xeF{=^#BpUk>2 z<}iz6S?t0r6(~*@-ulDz(%1#$aN)Ok_M3TdVphX7dOe*8(~(1Zj^trAu>`uf7#G`$ z#o_CAVTZc6bj4*Hb0=g!#s0DorkBT%WdixNFZiR6eG_263xsdb;{lPPSLt;t{G=3o zKE_9Zhhzl93yhWSIrgVJVVVlv4N#D4g;+ffp0IBhA2ZW?LLdg8>#}GU*elSD#axAm zsaS%<{S{t~qdiRphBJ$Qf8;1c)se+Nm)P%@>?}@exJIu>i2O9kqSZ~S3EV3i(gqpO zhT8>*{`>~oVCF%@1@h-rN8V}#aFBqY(k`xo@$_>?!=w)b; zyK~TW6upE;MA6H5lsh z*45}{Bx*{A3IwyS>#*wfq)rc(F*0wpVak!Iil_0*FJWQnJ0r?fTkW)#nx^W?*)+|9 zb=DiTvAJm9)Z~_8A?O?J(S*@LO;zq;WQN+46saeH_{*qjg1BbZ@!r#u%+|x`r31_~ zil~IQPl2#VE)1I4+ewfZOd*(%9RP56E;j*(?Nk)anjj>XiJIP$JdHRQH<^)V%@C5y z%4XrH%fKY^=Is+woArT^TqfQlQl#cPs?=+B zNGa+X>PVdtuEh38>U3H|Y^>KTdWqWB9Zc>%e4>{Wv&56n!p|4>tTnue7Qz>Je#pZ< z7>N=X$tHML;FlU!#N>IUe8ls)80=4dDj0gdnk^;DTz-HKN`|$}_-w(ABD9vWd82 z8yPauS1J{=hbMKd=E5$IKt9uh_>FOaUzSZn5=-4`W_RL?VC-uoE*^Lx#<6^}K=|Nu zPd?CV%7H!v{5uf4HZlub-V19s8AsWR-NQ7_p#y_P{PMscT#KPrgWepO1*Ws-f#BTdW}?)3beQk~%U|P*YGhaYt^F|XM*YWHORy1aYjL+>klmDnvX7?!LD2CH7YFvO9e<6ao|eH@ijNoTJi zF891D=5N4Z_xpBEhZqfH&FV3f)iv43j8v<}hpLzv$;R%sm5-_}oB1~45~Tl*P73{_ zbYdi!5Z83>3a3#T$(%kwSZ7)o9X5sjk(S%V=N+7*Jb#R5X>uqw@H9iFT|9;Jk>~fC za0owy$$kb7RUnc>luYr`IyF-Uo;8m_H2z{NpT0p}$MPpwo|wZ&JkM+6b)iCxSk&>+ z4+40a#jiaXc0TA#|$9 zjxqIcTK)CzStQi+H&b^N<`GTB>-zU+aS0eiVuESli|2Kb;>ZVp=3xlCG{Z>-+a`}! z@gekCD82jue8)HCcUofzD$gBK3>|yXKp4Sj6 z5S)XRlfcRk!LJ3gTUbVWX_*6rZhBDG{j5b{ZmW})DG%4q?Id&PfZUU=)#RSBN$(Uy+{^LYr6a&lvEQ#9`1;~AKe z$MTlkNj*c#O?jtnn+MI@GNh2zI&_jc3e&S=+t@5k`9Hd9AH4Us=H~B2;JGKj?BFcXWngC*pJ{x(PM{>uay%+_G)zTWoZ4-oHh!dAC6Mfao3GpM!5Zbm=g5 Date: Tue, 18 Jun 2024 22:04:32 +0800 Subject: [PATCH 15/54] Support to unparse `ScalarValue::IntervalMonthDayNano` to String (#10956) * support to unparse ScalarValue::IntervalMonthDayNano to String * use array formatter to format the interval string --- datafusion/expr/src/expr_fn.rs | 8 ++++- datafusion/sql/src/unparser/expr.rs | 49 ++++++++++++++++++++++++++--- 2 files changed, 52 insertions(+), 5 deletions(-) diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 099851aece46..a87412ee6356 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -31,8 +31,9 @@ use crate::{ Signature, Volatility, }; use crate::{AggregateUDFImpl, ColumnarValue, ScalarUDFImpl, WindowUDF, WindowUDFImpl}; +use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; use arrow::datatypes::{DataType, Field}; -use datafusion_common::{Column, Result}; +use datafusion_common::{Column, Result, ScalarValue}; use std::any::Any; use std::fmt::Debug; use std::ops::Not; @@ -670,6 +671,11 @@ impl WindowUDFImpl for SimpleWindowUDF { } } +pub fn interval_month_day_nano_lit(value: &str) -> Expr { + let interval = parse_interval_month_day_nano(value).ok(); + Expr::Literal(ScalarValue::IntervalMonthDayNano(interval)) +} + #[cfg(test)] mod test { use super::*; diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 1d197f3a0d8a..66334b0b410d 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. +use arrow::util::display::array_value_to_string; use core::fmt; use std::{fmt::Display, vec}; use arrow_array::{Date32Array, Date64Array}; use arrow_schema::DataType; +use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ - self, Expr as AstExpr, Function, FunctionArg, Ident, UnaryOperator, + self, Expr as AstExpr, Function, FunctionArg, Ident, Interval, UnaryOperator, }; use datafusion_common::{ @@ -834,7 +836,22 @@ impl Unparser<'_> { } ScalarValue::IntervalDayTime(None) => Ok(ast::Expr::Value(ast::Value::Null)), ScalarValue::IntervalMonthDayNano(Some(_i)) => { - not_impl_err!("Unsupported scalar: {v:?}") + let wrap_array = v.to_array()?; + let Some(result) = array_value_to_string(&wrap_array, 0).ok() else { + return internal_err!( + "Unable to convert IntervalMonthDayNano to string" + ); + }; + let interval = Interval { + value: Box::new(ast::Expr::Value(SingleQuotedString( + result.to_uppercase(), + ))), + leading_field: None, + leading_precision: None, + last_field: None, + fractional_seconds_precision: None, + }; + Ok(ast::Expr::Interval(interval)) } ScalarValue::IntervalMonthDayNano(None) => { Ok(ast::Expr::Value(ast::Value::Null)) @@ -962,19 +979,19 @@ impl Unparser<'_> { #[cfg(test)] mod tests { + use std::ops::{Add, Sub}; use std::{any::Any, sync::Arc, vec}; use arrow::datatypes::{Field, Schema}; use arrow_schema::DataType::Int8; - use datafusion_common::TableReference; - use datafusion_expr::AggregateExt; use datafusion_expr::{ case, col, cube, exists, grouping_set, lit, not, not_exists, out_ref_col, placeholder, rollup, table_scan, try_cast, when, wildcard, ColumnarValue, ScalarUDF, ScalarUDFImpl, Signature, Volatility, WindowFrame, WindowFunctionDefinition, }; + use datafusion_expr::{interval_month_day_nano_lit, AggregateExt}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::expr_fn::sum; @@ -1264,6 +1281,30 @@ mod tests { ), (col("need-quoted").eq(lit(1)), r#"("need-quoted" = 1)"#), (col("need quoted").eq(lit(1)), r#"("need quoted" = 1)"#), + ( + interval_month_day_nano_lit( + "1 YEAR 1 MONTH 1 DAY 3 HOUR 10 MINUTE 20 SECOND", + ), + r#"INTERVAL '0 YEARS 13 MONS 1 DAYS 3 HOURS 10 MINS 20.000000000 SECS'"#, + ), + ( + interval_month_day_nano_lit("1.5 MONTH"), + r#"INTERVAL '0 YEARS 1 MONS 15 DAYS 0 HOURS 0 MINS 0.000000000 SECS'"#, + ), + ( + interval_month_day_nano_lit("-3 MONTH"), + r#"INTERVAL '0 YEARS -3 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS'"#, + ), + ( + interval_month_day_nano_lit("1 MONTH") + .add(interval_month_day_nano_lit("1 DAY")), + r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' + INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, + ), + ( + interval_month_day_nano_lit("1 MONTH") + .sub(interval_month_day_nano_lit("1 DAY")), + r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' - INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, + ), ]; for (expr, expected) in tests { From a873f5156364f4357592c4bc9117887916e606f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 18 Jun 2024 22:08:13 +0800 Subject: [PATCH 16/54] Convert `StringAgg` to UDAF (#10945) * Convert StringAgg to UDAF * generate proto code * Fix bug * Fix * Add license * Add doc * Fix clippy * Remove aliases field * Add StringAgg proto test * Add roundtrip_expr_api test --- datafusion/expr/src/aggregate_function.rs | 8 - .../expr/src/type_coercion/aggregates.rs | 26 -- datafusion/functions-aggregate/src/lib.rs | 2 + .../functions-aggregate/src/string_agg.rs | 153 +++++++++++ .../physical-expr/src/aggregate/build_in.rs | 16 -- datafusion/physical-expr/src/aggregate/mod.rs | 1 - .../physical-expr/src/aggregate/string_agg.rs | 246 ------------------ .../physical-expr/src/expressions/mod.rs | 1 - datafusion/proto/proto/datafusion.proto | 2 +- datafusion/proto/src/generated/pbjson.rs | 3 - datafusion/proto/src/generated/prost.rs | 4 +- .../proto/src/logical_plan/from_proto.rs | 1 - datafusion/proto/src/logical_plan/to_proto.rs | 4 - .../proto/src/physical_plan/to_proto.rs | 5 +- .../tests/cases/roundtrip_logical_plan.rs | 2 + .../tests/cases/roundtrip_physical_plan.rs | 23 +- .../sqllogictest/test_files/aggregate.slt | 16 ++ 17 files changed, 192 insertions(+), 321 deletions(-) create mode 100644 datafusion/functions-aggregate/src/string_agg.rs delete mode 100644 datafusion/physical-expr/src/aggregate/string_agg.rs diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index a7fbf26febb1..1cde1c5050a8 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -51,8 +51,6 @@ pub enum AggregateFunction { BoolAnd, /// Bool Or BoolOr, - /// String aggregation - StringAgg, } impl AggregateFunction { @@ -68,7 +66,6 @@ impl AggregateFunction { Grouping => "GROUPING", BoolAnd => "BOOL_AND", BoolOr => "BOOL_OR", - StringAgg => "STRING_AGG", } } } @@ -92,7 +89,6 @@ impl FromStr for AggregateFunction { "min" => AggregateFunction::Min, "array_agg" => AggregateFunction::ArrayAgg, "nth_value" => AggregateFunction::NthValue, - "string_agg" => AggregateFunction::StringAgg, // statistical "corr" => AggregateFunction::Correlation, // other @@ -146,7 +142,6 @@ impl AggregateFunction { )))), AggregateFunction::Grouping => Ok(DataType::Int32), AggregateFunction::NthValue => Ok(coerced_data_types[0].clone()), - AggregateFunction::StringAgg => Ok(DataType::LargeUtf8), } } } @@ -195,9 +190,6 @@ impl AggregateFunction { AggregateFunction::Correlation => { Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable) } - AggregateFunction::StringAgg => { - Signature::uniform(2, STRINGS.to_vec(), Volatility::Immutable) - } } } } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index a216c98899fe..abe6d8b1823d 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -145,23 +145,6 @@ pub fn coerce_types( } AggregateFunction::NthValue => Ok(input_types.to_vec()), AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]), - AggregateFunction::StringAgg => { - if !is_string_agg_supported_arg_type(&input_types[0]) { - return plan_err!( - "The function {:?} does not support inputs of type {:?}", - agg_fun, - input_types[0] - ); - } - if !is_string_agg_supported_arg_type(&input_types[1]) { - return plan_err!( - "The function {:?} does not support inputs of type {:?}", - agg_fun, - input_types[1] - ); - } - Ok(vec![LargeUtf8, input_types[1].clone()]) - } } } @@ -391,15 +374,6 @@ pub fn is_integer_arg_type(arg_type: &DataType) -> bool { arg_type.is_integer() } -/// Return `true` if `arg_type` is of a [`DataType`] that the -/// [`AggregateFunction::StringAgg`] aggregation can operate on. -pub fn is_string_agg_supported_arg_type(arg_type: &DataType) -> bool { - matches!( - arg_type, - DataType::Utf8 | DataType::LargeUtf8 | DataType::Null - ) -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 990303bd1de3..20a8d2c15926 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -70,6 +70,7 @@ pub mod approx_median; pub mod approx_percentile_cont; pub mod approx_percentile_cont_with_weight; pub mod bit_and_or_xor; +pub mod string_agg; use crate::approx_percentile_cont::approx_percentile_cont_udaf; use crate::approx_percentile_cont_with_weight::approx_percentile_cont_with_weight_udaf; @@ -138,6 +139,7 @@ pub fn all_default_aggregate_functions() -> Vec> { approx_distinct::approx_distinct_udaf(), approx_percentile_cont_udaf(), approx_percentile_cont_with_weight_udaf(), + string_agg::string_agg_udaf(), bit_and_or_xor::bit_and_udaf(), bit_and_or_xor::bit_or_udaf(), bit_and_or_xor::bit_xor_udaf(), diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs new file mode 100644 index 000000000000..371cc8fb9739 --- /dev/null +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -0,0 +1,153 @@ +// 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. + +//! [`StringAgg`] and [`StringAggAccumulator`] accumulator for the `string_agg` function + +use arrow::array::ArrayRef; +use arrow_schema::DataType; +use datafusion_common::cast::as_generic_string_array; +use datafusion_common::Result; +use datafusion_common::{not_impl_err, ScalarValue}; +use datafusion_expr::function::AccumulatorArgs; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, Expr, Signature, TypeSignature, Volatility, +}; +use std::any::Any; + +make_udaf_expr_and_func!( + StringAgg, + string_agg, + expr delimiter, + "Concatenates the values of string expressions and places separator values between them", + string_agg_udaf +); + +/// STRING_AGG aggregate expression +#[derive(Debug)] +pub struct StringAgg { + signature: Signature, +} + +impl StringAgg { + /// Create a new StringAgg aggregate function + pub fn new() -> Self { + Self { + signature: Signature::one_of( + vec![ + TypeSignature::Exact(vec![DataType::LargeUtf8, DataType::Utf8]), + TypeSignature::Exact(vec![DataType::LargeUtf8, DataType::LargeUtf8]), + TypeSignature::Exact(vec![DataType::LargeUtf8, DataType::Null]), + ], + Volatility::Immutable, + ), + } + } +} + +impl Default for StringAgg { + fn default() -> Self { + Self::new() + } +} + +impl AggregateUDFImpl for StringAgg { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "string_agg" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::LargeUtf8) + } + + fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { + match &acc_args.input_exprs[1] { + Expr::Literal(ScalarValue::Utf8(Some(delimiter))) + | Expr::Literal(ScalarValue::LargeUtf8(Some(delimiter))) => { + Ok(Box::new(StringAggAccumulator::new(delimiter))) + } + Expr::Literal(ScalarValue::Utf8(None)) + | Expr::Literal(ScalarValue::LargeUtf8(None)) + | Expr::Literal(ScalarValue::Null) => { + Ok(Box::new(StringAggAccumulator::new(""))) + } + _ => not_impl_err!( + "StringAgg not supported for delimiter {}", + &acc_args.input_exprs[1] + ), + } + } +} + +#[derive(Debug)] +pub(crate) struct StringAggAccumulator { + values: Option, + delimiter: String, +} + +impl StringAggAccumulator { + pub fn new(delimiter: &str) -> Self { + Self { + values: None, + delimiter: delimiter.to_string(), + } + } +} + +impl Accumulator for StringAggAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let string_array: Vec<_> = as_generic_string_array::(&values[0])? + .iter() + .filter_map(|v| v.as_ref().map(ToString::to_string)) + .collect(); + if !string_array.is_empty() { + let s = string_array.join(self.delimiter.as_str()); + let v = self.values.get_or_insert("".to_string()); + if !v.is_empty() { + v.push_str(self.delimiter.as_str()); + } + v.push_str(s.as_str()); + } + Ok(()) + } + + fn merge_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + self.update_batch(values)?; + Ok(()) + } + + fn state(&mut self) -> Result> { + Ok(vec![self.evaluate()?]) + } + + fn evaluate(&mut self) -> Result { + Ok(ScalarValue::LargeUtf8(self.values.clone())) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + + self.values.as_ref().map(|v| v.capacity()).unwrap_or(0) + + self.delimiter.capacity() + } +} diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 6c01decdbf95..1dfe9ffd6905 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -155,22 +155,6 @@ pub fn create_aggregate_expr( ordering_req.to_vec(), )) } - (AggregateFunction::StringAgg, false) => { - if !ordering_req.is_empty() { - return not_impl_err!( - "STRING_AGG(ORDER BY a ASC) order-sensitive aggregations are not available" - ); - } - Arc::new(expressions::StringAgg::new( - input_phy_exprs[0].clone(), - input_phy_exprs[1].clone(), - name, - data_type, - )) - } - (AggregateFunction::StringAgg, true) => { - return not_impl_err!("STRING_AGG(DISTINCT) aggregations are not available"); - } }) } diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 0b1f5f577435..87c7deccc2cd 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -26,7 +26,6 @@ pub(crate) mod correlation; pub(crate) mod covariance; pub(crate) mod grouping; pub(crate) mod nth_value; -pub(crate) mod string_agg; #[macro_use] pub(crate) mod min_max; pub(crate) mod groups_accumulator; diff --git a/datafusion/physical-expr/src/aggregate/string_agg.rs b/datafusion/physical-expr/src/aggregate/string_agg.rs deleted file mode 100644 index dc0ffc557968..000000000000 --- a/datafusion/physical-expr/src/aggregate/string_agg.rs +++ /dev/null @@ -1,246 +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. - -//! [`StringAgg`] and [`StringAggAccumulator`] accumulator for the `string_agg` function - -use crate::aggregate::utils::down_cast_any_ref; -use crate::expressions::{format_state_name, Literal}; -use crate::{AggregateExpr, PhysicalExpr}; -use arrow::array::ArrayRef; -use arrow::datatypes::{DataType, Field}; -use datafusion_common::cast::as_generic_string_array; -use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_expr::Accumulator; -use std::any::Any; -use std::sync::Arc; - -/// STRING_AGG aggregate expression -#[derive(Debug)] -pub struct StringAgg { - name: String, - data_type: DataType, - expr: Arc, - delimiter: Arc, - nullable: bool, -} - -impl StringAgg { - /// Create a new StringAgg aggregate function - pub fn new( - expr: Arc, - delimiter: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - data_type, - delimiter, - expr, - nullable: true, - } - } -} - -impl AggregateExpr for StringAgg { - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - if let Some(delimiter) = self.delimiter.as_any().downcast_ref::() { - match delimiter.value() { - ScalarValue::Utf8(Some(delimiter)) - | ScalarValue::LargeUtf8(Some(delimiter)) => { - return Ok(Box::new(StringAggAccumulator::new(delimiter))); - } - ScalarValue::Null => { - return Ok(Box::new(StringAggAccumulator::new(""))); - } - _ => return not_impl_err!("StringAgg not supported for {}", self.name), - } - } - not_impl_err!("StringAgg not supported for {}", self.name) - } - - fn state_fields(&self) -> Result> { - Ok(vec![Field::new( - format_state_name(&self.name, "string_agg"), - self.data_type.clone(), - self.nullable, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone(), self.delimiter.clone()] - } - - fn name(&self) -> &str { - &self.name - } -} - -impl PartialEq for StringAgg { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.expr.eq(&x.expr) - && self.delimiter.eq(&x.delimiter) - }) - .unwrap_or(false) - } -} - -#[derive(Debug)] -pub(crate) struct StringAggAccumulator { - values: Option, - delimiter: String, -} - -impl StringAggAccumulator { - pub fn new(delimiter: &str) -> Self { - Self { - values: None, - delimiter: delimiter.to_string(), - } - } -} - -impl Accumulator for StringAggAccumulator { - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let string_array: Vec<_> = as_generic_string_array::(&values[0])? - .iter() - .filter_map(|v| v.as_ref().map(ToString::to_string)) - .collect(); - if !string_array.is_empty() { - let s = string_array.join(self.delimiter.as_str()); - let v = self.values.get_or_insert("".to_string()); - if !v.is_empty() { - v.push_str(self.delimiter.as_str()); - } - v.push_str(s.as_str()); - } - Ok(()) - } - - fn merge_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - self.update_batch(values)?; - Ok(()) - } - - fn state(&mut self) -> Result> { - Ok(vec![self.evaluate()?]) - } - - fn evaluate(&mut self) -> Result { - Ok(ScalarValue::LargeUtf8(self.values.clone())) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - + self.values.as_ref().map(|v| v.capacity()).unwrap_or(0) - + self.delimiter.capacity() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::tests::aggregate; - use crate::expressions::{col, create_aggregate_expr, try_cast}; - use arrow::datatypes::*; - use arrow::record_batch::RecordBatch; - use arrow_array::LargeStringArray; - use arrow_array::StringArray; - use datafusion_expr::type_coercion::aggregates::coerce_types; - use datafusion_expr::AggregateFunction; - - fn assert_string_aggregate( - array: ArrayRef, - function: AggregateFunction, - distinct: bool, - expected: ScalarValue, - delimiter: String, - ) { - let data_type = array.data_type(); - let sig = function.signature(); - let coerced = - coerce_types(&function, &[data_type.clone(), DataType::Utf8], &sig).unwrap(); - - let input_schema = Schema::new(vec![Field::new("a", data_type.clone(), true)]); - let batch = - RecordBatch::try_new(Arc::new(input_schema.clone()), vec![array]).unwrap(); - - let input = try_cast( - col("a", &input_schema).unwrap(), - &input_schema, - coerced[0].clone(), - ) - .unwrap(); - - let delimiter = Arc::new(Literal::new(ScalarValue::from(delimiter))); - let schema = Schema::new(vec![Field::new("a", coerced[0].clone(), true)]); - let agg = create_aggregate_expr( - &function, - distinct, - &[input, delimiter], - &[], - &schema, - "agg", - false, - ) - .unwrap(); - - let result = aggregate(&batch, agg).unwrap(); - assert_eq!(expected, result); - } - - #[test] - fn string_agg_utf8() { - let a: ArrayRef = Arc::new(StringArray::from(vec!["h", "e", "l", "l", "o"])); - assert_string_aggregate( - a, - AggregateFunction::StringAgg, - false, - ScalarValue::LargeUtf8(Some("h,e,l,l,o".to_owned())), - ",".to_owned(), - ); - } - - #[test] - fn string_agg_largeutf8() { - let a: ArrayRef = Arc::new(LargeStringArray::from(vec!["h", "e", "l", "l", "o"])); - assert_string_aggregate( - a, - AggregateFunction::StringAgg, - false, - ScalarValue::LargeUtf8(Some("h|e|l|l|o".to_owned())), - "|".to_owned(), - ); - } -} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index bffaafd7dac2..322610404074 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -47,7 +47,6 @@ pub use crate::aggregate::grouping::Grouping; pub use crate::aggregate::min_max::{Max, MaxAccumulator, Min, MinAccumulator}; pub use crate::aggregate::nth_value::NthValueAgg; pub use crate::aggregate::stats::StatsType; -pub use crate::aggregate::string_agg::StringAgg; pub use crate::window::cume_dist::{cume_dist, CumeDist}; pub use crate::window::lead_lag::{lag, lead, WindowShift}; pub use crate::window::nth_value::NthValue; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index ae4445eaa8ce..6375df721ae6 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -505,7 +505,7 @@ enum AggregateFunction { // REGR_SXX = 32; // REGR_SYY = 33; // REGR_SXY = 34; - STRING_AGG = 35; + // STRING_AGG = 35; NTH_VALUE_AGG = 36; } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 243c75435f8d..5c483f70d150 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -540,7 +540,6 @@ impl serde::Serialize for AggregateFunction { Self::Grouping => "GROUPING", Self::BoolAnd => "BOOL_AND", Self::BoolOr => "BOOL_OR", - Self::StringAgg => "STRING_AGG", Self::NthValueAgg => "NTH_VALUE_AGG", }; serializer.serialize_str(variant) @@ -561,7 +560,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "GROUPING", "BOOL_AND", "BOOL_OR", - "STRING_AGG", "NTH_VALUE_AGG", ]; @@ -611,7 +609,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "GROUPING" => Ok(AggregateFunction::Grouping), "BOOL_AND" => Ok(AggregateFunction::BoolAnd), "BOOL_OR" => Ok(AggregateFunction::BoolOr), - "STRING_AGG" => Ok(AggregateFunction::StringAgg), "NTH_VALUE_AGG" => Ok(AggregateFunction::NthValueAgg), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 1172eccb90fd..bc5b6be2ad87 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1959,7 +1959,7 @@ pub enum AggregateFunction { /// REGR_SXX = 32; /// REGR_SYY = 33; /// REGR_SXY = 34; - StringAgg = 35, + /// STRING_AGG = 35; NthValueAgg = 36, } impl AggregateFunction { @@ -1977,7 +1977,6 @@ impl AggregateFunction { AggregateFunction::Grouping => "GROUPING", AggregateFunction::BoolAnd => "BOOL_AND", AggregateFunction::BoolOr => "BOOL_OR", - AggregateFunction::StringAgg => "STRING_AGG", AggregateFunction::NthValueAgg => "NTH_VALUE_AGG", } } @@ -1992,7 +1991,6 @@ impl AggregateFunction { "GROUPING" => Some(Self::Grouping), "BOOL_AND" => Some(Self::BoolAnd), "BOOL_OR" => Some(Self::BoolOr), - "STRING_AGG" => Some(Self::StringAgg), "NTH_VALUE_AGG" => Some(Self::NthValueAgg), _ => None, } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 43cc352f98dd..5bec655bb1ff 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -146,7 +146,6 @@ impl From for AggregateFunction { protobuf::AggregateFunction::Correlation => Self::Correlation, protobuf::AggregateFunction::Grouping => Self::Grouping, protobuf::AggregateFunction::NthValueAgg => Self::NthValue, - protobuf::AggregateFunction::StringAgg => Self::StringAgg, } } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 33a58daeaf0a..66b7c77799ea 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -117,7 +117,6 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::Correlation => Self::Correlation, AggregateFunction::Grouping => Self::Grouping, AggregateFunction::NthValue => Self::NthValueAgg, - AggregateFunction::StringAgg => Self::StringAgg, } } } @@ -387,9 +386,6 @@ pub fn serialize_expr( AggregateFunction::NthValue => { protobuf::AggregateFunction::NthValueAgg } - AggregateFunction::StringAgg => { - protobuf::AggregateFunction::StringAgg - } }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 886179bf5627..ed966509b842 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -26,8 +26,7 @@ use datafusion::physical_plan::expressions::{ ArrayAgg, Avg, BinaryExpr, BoolAnd, BoolOr, CaseExpr, CastExpr, Column, Correlation, CumeDist, DistinctArrayAgg, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, Max, Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, - OrderSensitiveArrayAgg, Rank, RankType, RowNumber, StringAgg, TryCastExpr, - WindowShift, + OrderSensitiveArrayAgg, Rank, RankType, RowNumber, TryCastExpr, WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; @@ -260,8 +259,6 @@ fn aggr_expr_to_aggr_fn(expr: &dyn AggregateExpr) -> Result { protobuf::AggregateFunction::Avg } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::Correlation - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::StringAgg } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::NthValueAgg } else { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 52696a106183..61764394ee74 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -60,6 +60,7 @@ use datafusion_expr::{ WindowFunctionDefinition, WindowUDF, WindowUDFImpl, }; use datafusion_functions_aggregate::expr_fn::{bit_and, bit_or, bit_xor}; +use datafusion_functions_aggregate::string_agg::string_agg; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, logical_plan_to_bytes, logical_plan_to_bytes_with_extension_codec, @@ -669,6 +670,7 @@ async fn roundtrip_expr_api() -> Result<()> { bit_and(lit(2)), bit_or(lit(2)), bit_xor(lit(2)), + string_agg(col("a").cast_to(&DataType::Utf8, &schema)?, lit("|")), ]; // ensure expressions created with the expr api can be round tripped diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 7f66cdbf7663..eb3313239544 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -48,7 +48,7 @@ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ binary, cast, col, in_list, like, lit, Avg, BinaryExpr, Column, NotExpr, NthValue, - PhysicalSortExpr, StringAgg, + PhysicalSortExpr, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::DataSinkExec; @@ -79,6 +79,7 @@ use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ColumnarValue, ScalarUDF, ScalarUDFImpl, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, }; +use datafusion_functions_aggregate::string_agg::StringAgg; use datafusion_proto::physical_plan::{ AsExecutionPlan, DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, }; @@ -357,12 +358,20 @@ fn rountrip_aggregate() -> Result<()> { Vec::new(), ))], // STRING_AGG - vec![Arc::new(StringAgg::new( - cast(col("b", &schema)?, &schema, DataType::Utf8)?, - lit(ScalarValue::Utf8(Some(",".to_string()))), - "STRING_AGG(name, ',')".to_string(), - DataType::Utf8, - ))], + vec![udaf::create_aggregate_expr( + &AggregateUDF::new_from_impl(StringAgg::new()), + &[ + cast(col("b", &schema)?, &schema, DataType::Utf8)?, + lit(ScalarValue::Utf8(Some(",".to_string()))), + ], + &[], + &[], + &[], + &schema, + "STRING_AGG(name, ',')", + false, + false, + )?], ]; for aggregates in test_cases { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 0a6def3d6f27..378cab206240 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -4972,6 +4972,22 @@ CREATE TABLE float_table ( ( 32768.3, arrow_cast('NAN','Float32'), 32768.3, 32768.3 ), ( 27.3, 27.3, 27.3, arrow_cast('NAN','Float64') ); +# Test string_agg with largeutf8 +statement ok +create table string_agg_large_utf8 (c string) as values + (arrow_cast('a', 'LargeUtf8')), + (arrow_cast('b', 'LargeUtf8')), + (arrow_cast('c', 'LargeUtf8')) +; + +query T +SELECT STRING_AGG(c, ',') FROM string_agg_large_utf8; +---- +a,b,c + +statement ok +drop table string_agg_large_utf8; + query RRRRI select min(col_f32), max(col_f32), avg(col_f32), sum(col_f32), count(col_f32) from float_table; ---- From a2c9d1a8ba4445dec2a37df5a3fdd018158b91a6 Mon Sep 17 00:00:00 2001 From: Marvin Lanhenke <62298609+marvinlanhenke@users.noreply.github.com> Date: Tue, 18 Jun 2024 16:36:00 +0200 Subject: [PATCH 17/54] Minor: Return option from row_group_row_count (#10973) * refactor: return Option from row_group_row_count * fix: doctest --- datafusion-examples/examples/parquet_index.rs | 6 ++++- datafusion/core/benches/parquet_statistic.rs | 3 +-- .../physical_plan/parquet/row_groups.rs | 6 +++-- .../physical_plan/parquet/statistics.rs | 20 ++++++++++++--- .../core/tests/parquet/arrow_statistics.rs | 25 +++---------------- 5 files changed, 30 insertions(+), 30 deletions(-) diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index e3387117c91f..668eda047444 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -526,7 +526,11 @@ impl ParquetMetadataIndexBuilder { reader.schema(), reader.parquet_schema(), )?; - let row_counts = StatisticsConverter::row_group_row_counts(row_groups.iter())?; + let row_counts = converter + .row_group_row_counts(row_groups.iter())? + .ok_or_else(|| { + internal_datafusion_err!("Row group row counts are missing") + })?; let value_column_mins = converter.row_group_mins(row_groups.iter())?; let value_column_maxes = converter.row_group_maxes(row_groups.iter())?; diff --git a/datafusion/core/benches/parquet_statistic.rs b/datafusion/core/benches/parquet_statistic.rs index 5fd6b0066eb2..b58ecc13aee0 100644 --- a/datafusion/core/benches/parquet_statistic.rs +++ b/datafusion/core/benches/parquet_statistic.rs @@ -175,8 +175,7 @@ fn criterion_benchmark(c: &mut Criterion) { let _ = converter.row_group_mins(row_groups.iter()).unwrap(); let _ = converter.row_group_maxes(row_groups.iter()).unwrap(); let _ = converter.row_group_null_counts(row_groups.iter()).unwrap(); - let _ = StatisticsConverter::row_group_row_counts(row_groups.iter()) - .unwrap(); + let _ = converter.row_group_row_counts(row_groups.iter()).unwrap(); }) }, ); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index f8e4889f0b7f..e590f372253c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -384,10 +384,12 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { .map(|counts| Arc::new(counts) as ArrayRef) } - fn row_counts(&self, _column: &Column) -> Option { + fn row_counts(&self, column: &Column) -> Option { // row counts are the same for all columns in a row group - StatisticsConverter::row_group_row_counts(self.metadata_iter()) + self.statistics_converter(column) + .and_then(|c| c.row_group_row_counts(self.metadata_iter())) .ok() + .flatten() .map(|counts| Arc::new(counts) as ArrayRef) } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index 14d7bc2af42d..6ad78a82b9bf 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -718,21 +718,33 @@ impl<'a> StatisticsConverter<'a> { /// /// # Example /// ```no_run + /// # use arrow::datatypes::Schema; + /// # use arrow_array::ArrayRef; /// # use parquet::file::metadata::ParquetMetaData; /// # use datafusion::datasource::physical_plan::parquet::StatisticsConverter; /// # fn get_parquet_metadata() -> ParquetMetaData { unimplemented!() } - /// // Given the metadata for a parquet file + /// # fn get_arrow_schema() -> Schema { unimplemented!() } + /// // Given the metadata for a parquet file and the arrow schema /// let metadata: ParquetMetaData = get_parquet_metadata(); + /// let arrow_schema: Schema = get_arrow_schema(); + /// let parquet_schema = metadata.file_metadata().schema_descr(); + /// // create a converter + /// let converter = StatisticsConverter::try_new("foo", &arrow_schema, parquet_schema) + /// .unwrap(); /// // get the row counts for each row group - /// let row_counts = StatisticsConverter::row_group_row_counts(metadata + /// let row_counts = converter.row_group_row_counts(metadata /// .row_groups() /// .iter() /// ); /// ``` - pub fn row_group_row_counts(metadatas: I) -> Result + pub fn row_group_row_counts(&self, metadatas: I) -> Result> where I: IntoIterator, { + let Some(_) = self.parquet_index else { + return Ok(None); + }; + let mut builder = UInt64Array::builder(10); for metadata in metadatas.into_iter() { let row_count = metadata.num_rows(); @@ -743,7 +755,7 @@ impl<'a> StatisticsConverter<'a> { })?; builder.append_value(row_count); } - Ok(builder.finish()) + Ok(Some(builder.finish())) } /// Create a new `StatisticsConverter` to extract statistics for a column diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index cd0efc8d3525..4c68a57333e5 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -325,11 +325,9 @@ impl<'a> Test<'a> { Actual: {null_counts:?}. Expected: {expected_null_counts:?}" ); - let row_counts = StatisticsConverter::row_group_row_counts( - reader.metadata().row_groups().iter(), - ) - .unwrap(); - let row_counts = Some(row_counts); + let row_counts = converter + .row_group_row_counts(reader.metadata().row_groups().iter()) + .unwrap(); assert_eq!( row_counts, expected_row_counts, "{column_name}: Mismatch with expected row counts. \ @@ -2001,21 +1999,6 @@ async fn test_column_non_existent() { .build() .await; - Test { - reader: &reader, - // mins are [-5, -4, 0, 5] - expected_min: Arc::new(Int64Array::from(vec![None, None, None, None])), - // maxes are [-1, 0, 4, 9] - expected_max: Arc::new(Int64Array::from(vec![None, None, None, None])), - // nulls are [0, 0, 0, 0] - expected_null_counts: UInt64Array::from(vec![None, None, None, None]), - // row counts are [5, 5, 5, 5] - expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), - column_name: "i_do_not_exist", - check: Check::RowGroup, - } - .run_with_schema(&schema); - Test { reader: &reader, // mins are [-5, -4, 0, 5] @@ -2027,7 +2010,7 @@ async fn test_column_non_existent() { // row counts are [5, 5, 5, 5] expected_row_counts: None, column_name: "i_do_not_exist", - check: Check::DataPage, + check: Check::Both, } .run_with_schema(&schema); } From e9f9a239ae9467850b7d17c42f0f11555a7d3058 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Tue, 18 Jun 2024 10:25:53 -0700 Subject: [PATCH 18/54] Minor: Add routine to debug join fuzz tests (#10970) * Minor: Add routine to debug join fuzz tests --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 202 ++++++++++++++---- 1 file changed, 162 insertions(+), 40 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 516749e82a53..5fdf02079496 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -43,6 +43,17 @@ use datafusion::physical_plan::memory::MemoryExec; use datafusion::prelude::{SessionConfig, SessionContext}; use test_utils::stagger_batch_with_seed; +// Determines what Fuzz tests needs to run +// Ideally all tests should match, but in reality some tests +// passes only partial cases +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +enum JoinTestType { + // compare NestedLoopJoin and HashJoin + NljHj, + // compare HashJoin and SortMergeJoin, no need to compare SortMergeJoin and NestedLoopJoin + // because if existing variants both passed that means SortMergeJoin and NestedLoopJoin also passes + HjSmj, +} #[tokio::test] async fn test_inner_join_1k() { JoinFuzzTestCase::new( @@ -51,7 +62,7 @@ async fn test_inner_join_1k() { JoinType::Inner, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -71,6 +82,30 @@ fn less_than_100_join_filter(schema1: Arc, _schema2: Arc) -> Joi JoinFilter::new(less_than_100, column_indices, intermediate_schema) } +fn col_lt_col_filter(schema1: Arc, schema2: Arc) -> JoinFilter { + let less_than_100 = Arc::new(BinaryExpr::new( + Arc::new(Column::new("x", 1)), + Operator::Lt, + Arc::new(Column::new("x", 0)), + )) as _; + let column_indices = vec![ + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ColumnIndex { + index: 2, + side: JoinSide::Right, + }, + ]; + let intermediate_schema = Schema::new(vec![ + schema1.field_with_name("x").unwrap().to_owned(), + schema2.field_with_name("x").unwrap().to_owned(), + ]); + + JoinFilter::new(less_than_100, column_indices, intermediate_schema) +} + #[tokio::test] async fn test_inner_join_1k_filtered() { JoinFuzzTestCase::new( @@ -79,7 +114,7 @@ async fn test_inner_join_1k_filtered() { JoinType::Inner, Some(Box::new(less_than_100_join_filter)), ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -91,7 +126,7 @@ async fn test_inner_join_1k_smjoin() { JoinType::Inner, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -103,7 +138,7 @@ async fn test_left_join_1k() { JoinType::Left, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -115,7 +150,7 @@ async fn test_left_join_1k_filtered() { JoinType::Left, Some(Box::new(less_than_100_join_filter)), ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -127,7 +162,7 @@ async fn test_right_join_1k() { JoinType::Right, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } // Add support for Right filtered joins @@ -140,7 +175,7 @@ async fn test_right_join_1k_filtered() { JoinType::Right, Some(Box::new(less_than_100_join_filter)), ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -152,7 +187,7 @@ async fn test_full_join_1k() { JoinType::Full, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -164,7 +199,7 @@ async fn test_full_join_1k_filtered() { JoinType::Full, Some(Box::new(less_than_100_join_filter)), ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -176,12 +211,13 @@ async fn test_semi_join_1k() { JoinType::LeftSemi, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } // The test is flaky // https://github.com/apache/datafusion/issues/10886 +// SMJ produces 1 more row in the output #[ignore] #[tokio::test] async fn test_semi_join_1k_filtered() { @@ -189,9 +225,9 @@ async fn test_semi_join_1k_filtered() { make_staggered_batches(1000), make_staggered_batches(1000), JoinType::LeftSemi, - Some(Box::new(less_than_100_join_filter)), + Some(Box::new(col_lt_col_filter)), ) - .run_test() + .run_test(&[JoinTestType::HjSmj], false) .await } @@ -203,7 +239,7 @@ async fn test_anti_join_1k() { JoinType::LeftAnti, None, ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -217,7 +253,7 @@ async fn test_anti_join_1k_filtered() { JoinType::LeftAnti, Some(Box::new(less_than_100_join_filter)), ) - .run_test() + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -331,7 +367,7 @@ impl JoinFuzzTestCase { self.on_columns().clone(), self.join_filter(), self.join_type, - vec![SortOptions::default(), SortOptions::default()], + vec![SortOptions::default(); self.on_columns().len()], false, ) .unwrap(), @@ -381,9 +417,11 @@ impl JoinFuzzTestCase { ) } - /// Perform sort-merge join and hash join on same input - /// and verify two outputs are equal - async fn run_test(&self) { + /// Perform joins tests on same inputs and verify outputs are equal + /// `join_tests` - identifies what join types to test + /// if `debug` flag is set the test will save randomly generated inputs and outputs to user folders, + /// so it is easy to debug a test on top of the failed data + async fn run_test(&self, join_tests: &[JoinTestType], debug: bool) { for batch_size in self.batch_sizes { let session_config = SessionConfig::new().with_batch_size(*batch_size); let ctx = SessionContext::new_with_config(session_config); @@ -394,17 +432,30 @@ impl JoinFuzzTestCase { let hj = self.hash_join(); let hj_collected = collect(hj, task_ctx.clone()).await.unwrap(); + let nlj = self.nested_loop_join(); + let nlj_collected = collect(nlj, task_ctx.clone()).await.unwrap(); + // Get actual row counts(without formatting overhead) for HJ and SMJ let hj_rows = hj_collected.iter().fold(0, |acc, b| acc + b.num_rows()); let smj_rows = smj_collected.iter().fold(0, |acc, b| acc + b.num_rows()); + let nlj_rows = nlj_collected.iter().fold(0, |acc, b| acc + b.num_rows()); - assert_eq!( - hj_rows, smj_rows, - "SortMergeJoinExec and HashJoinExec produced different row counts" - ); + if debug { + println!("The debug is ON. Input data will be saved"); + let out_dir_name = &format!("fuzz_test_debug_batch_size_{batch_size}"); + Self::save_as_parquet(&self.input1, out_dir_name, "input1"); + Self::save_as_parquet(&self.input2, out_dir_name, "input2"); - let nlj = self.nested_loop_join(); - let nlj_collected = collect(nlj, task_ctx.clone()).await.unwrap(); + if join_tests.contains(&JoinTestType::NljHj) { + Self::save_as_parquet(&nlj_collected, out_dir_name, "nlj"); + Self::save_as_parquet(&hj_collected, out_dir_name, "hj"); + } + + if join_tests.contains(&JoinTestType::HjSmj) { + Self::save_as_parquet(&hj_collected, out_dir_name, "hj"); + Self::save_as_parquet(&smj_collected, out_dir_name, "smj"); + } + } // compare let smj_formatted = @@ -425,35 +476,106 @@ impl JoinFuzzTestCase { nlj_formatted.trim().lines().collect(); nlj_formatted_sorted.sort_unstable(); - // row level compare if any of joins returns the result - // the reason is different formatting when there is no rows - if smj_rows > 0 || hj_rows > 0 { - for (i, (smj_line, hj_line)) in smj_formatted_sorted + if join_tests.contains(&JoinTestType::NljHj) { + let err_msg_rowcnt = format!("NestedLoopJoinExec and HashJoinExec produced different row counts, batch_size: {}", batch_size); + assert_eq!(nlj_rows, hj_rows, "{}", err_msg_rowcnt.as_str()); + + let err_msg_contents = format!("NestedLoopJoinExec and HashJoinExec produced different results, batch_size: {}", batch_size); + // row level compare if any of joins returns the result + // the reason is different formatting when there is no rows + for (i, (nlj_line, hj_line)) in nlj_formatted_sorted .iter() .zip(&hj_formatted_sorted) .enumerate() { assert_eq!( - (i, smj_line), + (i, nlj_line), (i, hj_line), - "SortMergeJoinExec and HashJoinExec produced different results" + "{}", + err_msg_contents.as_str() ); } } - for (i, (nlj_line, hj_line)) in nlj_formatted_sorted - .iter() - .zip(&hj_formatted_sorted) - .enumerate() - { - assert_eq!( - (i, nlj_line), - (i, hj_line), - "NestedLoopJoinExec and HashJoinExec produced different results" - ); + if join_tests.contains(&JoinTestType::HjSmj) { + let err_msg_row_cnt = format!("HashJoinExec and SortMergeJoinExec produced different row counts, batch_size: {}", &batch_size); + assert_eq!(hj_rows, smj_rows, "{}", err_msg_row_cnt.as_str()); + + let err_msg_contents = format!("SortMergeJoinExec and HashJoinExec produced different results, batch_size: {}", &batch_size); + // row level compare if any of joins returns the result + // the reason is different formatting when there is no rows + if smj_rows > 0 || hj_rows > 0 { + for (i, (smj_line, hj_line)) in smj_formatted_sorted + .iter() + .zip(&hj_formatted_sorted) + .enumerate() + { + assert_eq!( + (i, smj_line), + (i, hj_line), + "{}", + err_msg_contents.as_str() + ); + } + } } } } + + /// This method useful for debugging fuzz tests + /// It helps to save randomly generated input test data for both join inputs into the user folder + /// as a parquet files preserving partitioning. + /// Once the data is saved it is possible to run a custom test on top of the saved data and debug + /// + /// let ctx: SessionContext = SessionContext::new(); + /// let df = ctx + /// .read_parquet( + /// "/tmp/input1/*.parquet", + /// ParquetReadOptions::default(), + /// ) + /// .await + /// .unwrap(); + /// let left = df.collect().await.unwrap(); + /// + /// let df = ctx + /// .read_parquet( + /// "/tmp/input2/*.parquet", + /// ParquetReadOptions::default(), + /// ) + /// .await + /// .unwrap(); + /// + /// let right = df.collect().await.unwrap(); + /// JoinFuzzTestCase::new( + /// left, + /// right, + /// JoinType::LeftSemi, + /// Some(Box::new(less_than_100_join_filter)), + /// ) + /// .run_test() + /// .await + /// } + fn save_as_parquet(input: &[RecordBatch], output_dir: &str, out_name: &str) { + let out_path = &format!("{output_dir}/{out_name}"); + std::fs::remove_dir_all(out_path).unwrap_or(()); + std::fs::create_dir_all(out_path).unwrap(); + + input.iter().enumerate().for_each(|(idx, batch)| { + let mut file = + std::fs::File::create(format!("{out_path}/file_{}.parquet", idx)) + .unwrap(); + let mut writer = parquet::arrow::ArrowWriter::try_new( + &mut file, + input.first().unwrap().schema(), + None, + ) + .expect("creating writer"); + writer.write(batch).unwrap(); + writer.close().unwrap(); + }); + + println!("The data {out_name} saved as parquet into {out_path}"); + } } /// Return randomly sized record batches with: From fbf793434e8f94fc08b55ad1ced68b10d3a8c2ff Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 19 Jun 2024 04:03:32 +0800 Subject: [PATCH 19/54] Support to unparse `ScalarValue::TimestampNanosecond` to String (#10984) * support unparse TimestampNanosecond * cargo fmt * extract the duplicate code --- datafusion/sql/src/unparser/expr.rs | 48 ++++++++++++++++++++++++++--- 1 file changed, 44 insertions(+), 4 deletions(-) diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 66334b0b410d..65481aed64f9 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -19,11 +19,12 @@ use arrow::util::display::array_value_to_string; use core::fmt; use std::{fmt::Display, vec}; -use arrow_array::{Date32Array, Date64Array}; +use arrow_array::{Date32Array, Date64Array, TimestampNanosecondArray}; use arrow_schema::DataType; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ - self, Expr as AstExpr, Function, FunctionArg, Ident, Interval, UnaryOperator, + self, Expr as AstExpr, Function, FunctionArg, Ident, Interval, TimezoneInfo, + UnaryOperator, }; use datafusion_common::{ @@ -819,8 +820,36 @@ impl Unparser<'_> { ScalarValue::TimestampMicrosecond(None, _) => { Ok(ast::Expr::Value(ast::Value::Null)) } - ScalarValue::TimestampNanosecond(Some(_ts), _) => { - not_impl_err!("Unsupported scalar: {v:?}") + ScalarValue::TimestampNanosecond(Some(_ts), tz) => { + let result = if let Some(tz) = tz { + v.to_array()? + .as_any() + .downcast_ref::() + .ok_or(internal_datafusion_err!( + "Unable to downcast to TimestampNanosecond from TimestampNanosecond scalar" + ))? + .value_as_datetime_with_tz(0, tz.parse()?) + .ok_or(internal_datafusion_err!( + "Unable to convert TimestampNanosecond to DateTime" + ))?.to_string() + } else { + v.to_array()? + .as_any() + .downcast_ref::() + .ok_or(internal_datafusion_err!( + "Unable to downcast to TimestampNanosecond from TimestampNanosecond scalar" + ))? + .value_as_datetime(0) + .ok_or(internal_datafusion_err!( + "Unable to convert TimestampNanosecond to NaiveDateTime" + ))?.to_string() + }; + Ok(ast::Expr::Cast { + kind: ast::CastKind::Cast, + expr: Box::new(ast::Expr::Value(SingleQuotedString(result))), + data_type: ast::DataType::Timestamp(None, TimezoneInfo::None), + format: None, + }) } ScalarValue::TimestampNanosecond(None, _) => { Ok(ast::Expr::Value(ast::Value::Null)) @@ -1151,6 +1180,17 @@ mod tests { Expr::Literal(ScalarValue::Date32(Some(-1))), r#"CAST('1969-12-31' AS DATE)"#, ), + ( + Expr::Literal(ScalarValue::TimestampNanosecond(Some(10001), None)), + r#"CAST('1970-01-01 00:00:00.000010001' AS TIMESTAMP)"#, + ), + ( + Expr::Literal(ScalarValue::TimestampNanosecond( + Some(10001), + Some("+08:00".into()), + )), + r#"CAST('1970-01-01 08:00:00.000010001 +08:00' AS TIMESTAMP)"#, + ), (sum(col("a")), r#"sum(a)"#), ( count_udaf() From 2a49d61658c56bb43ccc14970b30181a5c5daa3b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Jun 2024 16:27:41 -0400 Subject: [PATCH 20/54] build(deps-dev): bump ws in /datafusion/wasmtest/datafusion-wasm-app (#10988) Bumps [ws](https://github.com/websockets/ws) from 8.14.2 to 8.17.1. - [Release notes](https://github.com/websockets/ws/releases) - [Commits](https://github.com/websockets/ws/compare/8.14.2...8.17.1) --- updated-dependencies: - dependency-name: ws dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../wasmtest/datafusion-wasm-app/package-lock.json | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json index 4cfe0b5a0cd2..995984a13ebc 100644 --- a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json +++ b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json @@ -4323,9 +4323,9 @@ "dev": true }, "node_modules/ws": { - "version": "8.14.2", - "resolved": "https://registry.npmjs.org/ws/-/ws-8.14.2.tgz", - "integrity": "sha512-wEBG1ftX4jcglPxgFCMJmZ2PLtSbJ2Peg6TmpJFTbe9GZYOQCDPdMYu/Tm0/bGZkw8paZnJY45J4K2PZrLYq8g==", + "version": "8.17.1", + "resolved": "https://registry.npmjs.org/ws/-/ws-8.17.1.tgz", + "integrity": "sha512-6XQFvXTkbfUOZOKKILFG1PDK2NDQs4azKQl26T0YS5CxqWLgXajbPZ+h4gZekJyRqFU8pvnbAbbs/3TgRPy+GQ==", "dev": true, "engines": { "node": ">=10.0.0" @@ -7618,9 +7618,9 @@ "dev": true }, "ws": { - "version": "8.14.2", - "resolved": "https://registry.npmjs.org/ws/-/ws-8.14.2.tgz", - "integrity": "sha512-wEBG1ftX4jcglPxgFCMJmZ2PLtSbJ2Peg6TmpJFTbe9GZYOQCDPdMYu/Tm0/bGZkw8paZnJY45J4K2PZrLYq8g==", + "version": "8.17.1", + "resolved": "https://registry.npmjs.org/ws/-/ws-8.17.1.tgz", + "integrity": "sha512-6XQFvXTkbfUOZOKKILFG1PDK2NDQs4azKQl26T0YS5CxqWLgXajbPZ+h4gZekJyRqFU8pvnbAbbs/3TgRPy+GQ==", "dev": true, "requires": {} }, From 80f43224299bf05f18ffa3be3909303473b2cd36 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 18 Jun 2024 17:01:12 -0400 Subject: [PATCH 21/54] Minor: reuse Rows buffer in GroupValuesRows (#10980) --- .../src/aggregates/group_values/row.rs | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 3b7480cd292a..96a12d7b62da 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -59,9 +59,12 @@ pub struct GroupValuesRows { /// [`Row`]: arrow::row::Row group_values: Option, - // buffer to be reused to store hashes + /// reused buffer to store hashes hashes_buffer: Vec, + /// reused buffer to store rows + rows_buffer: Rows, + /// Random state for creating hashes random_state: RandomState, } @@ -78,6 +81,10 @@ impl GroupValuesRows { let map = RawTable::with_capacity(0); + let starting_rows_capacity = 1000; + let starting_data_capacity = 64 * starting_rows_capacity; + let rows_buffer = + row_converter.empty_rows(starting_rows_capacity, starting_data_capacity); Ok(Self { schema, row_converter, @@ -85,6 +92,7 @@ impl GroupValuesRows { map_size: 0, group_values: None, hashes_buffer: Default::default(), + rows_buffer, random_state: Default::default(), }) } @@ -93,8 +101,9 @@ impl GroupValuesRows { impl GroupValues for GroupValuesRows { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { // Convert the group keys into the row format - // Avoid reallocation when https://github.com/apache/arrow-rs/issues/4479 is available - let group_rows = self.row_converter.convert_columns(cols)?; + let group_rows = &mut self.rows_buffer; + group_rows.clear(); + self.row_converter.append(group_rows, cols)?; let n_rows = group_rows.num_rows(); let mut group_values = match self.group_values.take() { @@ -150,6 +159,7 @@ impl GroupValues for GroupValuesRows { self.row_converter.size() + group_values_size + self.map_size + + self.rows_buffer.size() + self.hashes_buffer.allocated_size() } From 5cb1917f8c911184b030599f0a3cb86e2d602031 Mon Sep 17 00:00:00 2001 From: Lorrens Pantelis <100197010+LorrensP-2158466@users.noreply.github.com> Date: Wed, 19 Jun 2024 01:46:54 +0200 Subject: [PATCH 22/54] Add example for writing SQL analysis using DataFusion structures (#10938) * sql analysis example * update examples readme * update comments * Update datafusion-examples/examples/sql_analysis.rs Co-authored-by: Andrew Lamb * apply feedback * Run tapelo to fix Cargo.toml formatting * Tweak comments --------- Co-authored-by: Andrew Lamb --- datafusion-examples/Cargo.toml | 1 + datafusion-examples/README.md | 1 + datafusion-examples/examples/sql_analysis.rs | 309 +++++++++++++++++++ 3 files changed, 311 insertions(+) create mode 100644 datafusion-examples/examples/sql_analysis.rs diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 0bcf7c1afc15..c96aa7ae3951 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -76,6 +76,7 @@ prost-derive = { version = "0.12", default-features = false } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } tempfile = { workspace = true } +test-utils = { path = "../test-utils" } tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] } tonic = "0.11" url = { workspace = true } diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index c34f706adb82..6150c551c900 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -74,6 +74,7 @@ cargo run --example csv_sql - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF) - [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined Scalar Function (UDF) - [`simple_udfw.rs`](examples/simple_udwf.rs): Define and invoke a User Defined Window Function (UDWF) +- [`sql_analysis.rs`](examples/sql_analysis.rs): Analyse SQL queries with DataFusion structures - [`sql_dialect.rs`](examples/sql_dialect.rs): Example of implementing a custom SQL dialect on top of `DFParser` - [`to_char.rs`](examples/to_char.rs): Examples of using the to_char function - [`to_timestamp.rs`](examples/to_timestamp.rs): Examples of using to_timestamp functions diff --git a/datafusion-examples/examples/sql_analysis.rs b/datafusion-examples/examples/sql_analysis.rs new file mode 100644 index 000000000000..3995988751c7 --- /dev/null +++ b/datafusion-examples/examples/sql_analysis.rs @@ -0,0 +1,309 @@ +// 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. + +//! This example shows how to use the structures that DataFusion provides to perform +//! Analysis on SQL queries and their plans. +//! +//! As a motivating example, we show how to count the number of JOINs in a query +//! as well as how many join tree's there are with their respective join count + +use std::sync::Arc; + +use datafusion::common::Result; +use datafusion::{ + datasource::MemTable, + execution::context::{SessionConfig, SessionContext}, +}; +use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; +use datafusion_expr::LogicalPlan; +use test_utils::tpcds::tpcds_schemas; + +/// Counts the total number of joins in a plan +fn total_join_count(plan: &LogicalPlan) -> usize { + let mut total = 0; + + // We can use the TreeNode API to walk over a LogicalPlan. + plan.apply(|node| { + // if we encounter a join we update the running count + if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { + total += 1; + } + Ok(TreeNodeRecursion::Continue) + }) + .unwrap(); + + total +} + +/// Counts the total number of joins in a plan and collects every join tree in +/// the plan with their respective join count. +/// +/// Join Tree Definition: the largest subtree consisting entirely of joins +/// +/// For example, this plan: +/// +/// ```text +/// JOIN +/// / \ +/// A JOIN +/// / \ +/// B C +/// ``` +/// +/// has a single join tree `(A-B-C)` which will result in `(2, [2])` +/// +/// This plan: +/// +/// ```text +/// JOIN +/// / \ +/// A GROUP +/// | +/// JOIN +/// / \ +/// B C +/// ``` +/// +/// Has two join trees `(A-, B-C)` which will result in `(2, [1, 1])` +fn count_trees(plan: &LogicalPlan) -> (usize, Vec) { + // this works the same way as `total_count`, but now when we encounter a Join + // we try to collect it's entire tree + let mut to_visit = vec![plan]; + let mut total = 0; + let mut groups = vec![]; + + while let Some(node) = to_visit.pop() { + // if we encouter a join, we know were at the root of the tree + // count this tree and recurse on it's inputs + if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { + let (group_count, inputs) = count_tree(node); + total += group_count; + groups.push(group_count); + to_visit.extend(inputs); + } else { + to_visit.extend(node.inputs()); + } + } + + (total, groups) +} + +/// Count the entire join tree and return its inputs using TreeNode API +/// +/// For example, if this function receives following plan: +/// +/// ```text +/// JOIN +/// / \ +/// A GROUP +/// | +/// JOIN +/// / \ +/// B C +/// ``` +/// +/// It will return `(1, [A, GROUP])` +fn count_tree(join: &LogicalPlan) -> (usize, Vec<&LogicalPlan>) { + let mut inputs = Vec::new(); + let mut total = 0; + + join.apply(|node| { + // Some extra knowledge: + // + // optimized plans have their projections pushed down as far as + // possible, which sometimes results in a projection going in between 2 + // subsequent joins giving the illusion these joins are not "related", + // when in fact they are. + // + // This plan: + // JOIN + // / \ + // A PROJECTION + // | + // JOIN + // / \ + // B C + // + // is the same as: + // + // JOIN + // / \ + // A JOIN + // / \ + // B C + // we can continue the recursion in this case + if let LogicalPlan::Projection(_) = node { + return Ok(TreeNodeRecursion::Continue); + } + + // any join we count + if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { + total += 1; + Ok(TreeNodeRecursion::Continue) + } else { + inputs.push(node); + // skip children of input node + Ok(TreeNodeRecursion::Jump) + } + }) + .unwrap(); + + (total, inputs) +} + +#[tokio::main] +async fn main() -> Result<()> { + // To show how we can count the joins in a sql query we'll be using query 88 + // from the TPC-DS benchmark. + // + // q8 has many joins, cross-joins and multiple join-trees, perfect for our + // example: + + let tpcds_query_88 = " +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s8;"; + + // first set up the config + let config = SessionConfig::default(); + let ctx = SessionContext::new_with_config(config); + + // register the tables of the TPC-DS query + let tables = tpcds_schemas(); + for table in tables { + ctx.register_table( + table.name, + Arc::new(MemTable::try_new(Arc::new(table.schema.clone()), vec![])?), + )?; + } + // We can create a LogicalPlan from a SQL query like this + let logical_plan = ctx.sql(tpcds_query_88).await?.into_optimized_plan()?; + + println!( + "Optimized Logical Plan:\n\n{}\n", + logical_plan.display_indent() + ); + // we can get the total count (query 88 has 31 joins: 7 CROSS joins and 24 INNER joins => 40 input relations) + let total_join_count = total_join_count(&logical_plan); + assert_eq!(31, total_join_count); + + println!("The plan has {total_join_count} joins."); + + // Furthermore the 24 inner joins are 8 groups of 3 joins with the 7 + // cross-joins combining them we can get these groups using the + // `count_trees` method + let (total_join_count, trees) = count_trees(&logical_plan); + assert_eq!( + (total_join_count, &trees), + // query 88 is very straightforward, we know the cross-join group is at + // the top of the plan followed by the INNER joins + (31, &vec![7, 3, 3, 3, 3, 3, 3, 3, 3]) + ); + + println!( + "And following join-trees (number represents join amount in tree): {trees:?}" + ); + + Ok(()) +} From 8fda4a6163970cd4ee02cc97468b2a1f67012ae2 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Wed, 19 Jun 2024 10:08:12 +0800 Subject: [PATCH 23/54] feat(optimizer): handle partial anchored regex cases and improve doc (#10977) Signed-off-by: Ruihang Xia --- .../simplify_expressions/expr_simplifier.rs | 12 +++- .../src/simplify_expressions/regex.rs | 68 ++++++++++++++++--- 2 files changed, 67 insertions(+), 13 deletions(-) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 024cb7440388..f2c80e4a7207 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -2730,11 +2730,10 @@ mod tests { // unsupported cases assert_no_change(regex_match(col("c1"), lit("foo.*"))); assert_no_change(regex_match(col("c1"), lit("(foo)"))); - assert_no_change(regex_match(col("c1"), lit("^foo"))); - assert_no_change(regex_match(col("c1"), lit("foo$"))); assert_no_change(regex_match(col("c1"), lit("%"))); assert_no_change(regex_match(col("c1"), lit("_"))); assert_no_change(regex_match(col("c1"), lit("f%o"))); + assert_no_change(regex_match(col("c1"), lit("^f%o"))); assert_no_change(regex_match(col("c1"), lit("f_o"))); // empty cases @@ -2827,13 +2826,20 @@ mod tests { assert_no_change(regex_match(col("c1"), lit("(foo|ba_r)*"))); assert_no_change(regex_match(col("c1"), lit("(fo_o|ba_r)*"))); assert_no_change(regex_match(col("c1"), lit("^(foo|bar)*"))); - assert_no_change(regex_match(col("c1"), lit("^foo|bar$"))); assert_no_change(regex_match(col("c1"), lit("^(foo)(bar)$"))); assert_no_change(regex_match(col("c1"), lit("^"))); assert_no_change(regex_match(col("c1"), lit("$"))); assert_no_change(regex_match(col("c1"), lit("$^"))); assert_no_change(regex_match(col("c1"), lit("$foo^"))); + // regular expressions that match a partial literal + assert_change(regex_match(col("c1"), lit("^foo")), like(col("c1"), "foo%")); + assert_change(regex_match(col("c1"), lit("foo$")), like(col("c1"), "%foo")); + assert_change( + regex_match(col("c1"), lit("^foo|bar$")), + like(col("c1"), "foo%").or(like(col("c1"), "%bar")), + ); + // OR-chain assert_change( regex_match(col("c1"), lit("foo|bar|baz")), diff --git a/datafusion/optimizer/src/simplify_expressions/regex.rs b/datafusion/optimizer/src/simplify_expressions/regex.rs index 175b70f2b10e..9a78298b10a7 100644 --- a/datafusion/optimizer/src/simplify_expressions/regex.rs +++ b/datafusion/optimizer/src/simplify_expressions/regex.rs @@ -22,6 +22,19 @@ use regex_syntax::hir::{Capture, Hir, HirKind, Literal, Look}; /// Maximum number of regex alternations (`foo|bar|...`) that will be expanded into multiple `LIKE` expressions. const MAX_REGEX_ALTERNATIONS_EXPANSION: usize = 4; +/// Tries to convert a regexp expression to a `LIKE` or `Eq`/`NotEq` expression. +/// +/// This function also validates the regex pattern. And will return error if the +/// pattern is invalid. +/// +/// Typical cases this function can simplify: +/// - empty regex pattern to `LIKE '%'` +/// - literal regex patterns to `LIKE '%foo%'` +/// - full anchored regex patterns (e.g. `^foo$`) to `= 'foo'` +/// - partial anchored regex patterns (e.g. `^foo`) to `LIKE 'foo%'` +/// - combinations (alternatives) of the above, will be concatenated with `OR` or `AND` +/// +/// Dev note: unit tests of this function are in `expr_simplifier.rs`, case `test_simplify_regex`. pub fn simplify_regex_expr( left: Box, op: Operator, @@ -53,13 +66,15 @@ pub fn simplify_regex_expr( } } - // leave untouched if optimization didn't work + // Leave untouched if optimization didn't work Ok(Expr::BinaryExpr(BinaryExpr { left, op, right })) } #[derive(Debug)] struct OperatorMode { + /// Negative match. not: bool, + /// Ignore case (`true` for case-insensitive). i: bool, } @@ -80,6 +95,7 @@ impl OperatorMode { Self { not, i } } + /// Creates an [`LIKE`](Expr::Like) from the given `LIKE` pattern. fn expr(&self, expr: Box, pattern: String) -> Expr { let like = Like { negated: self.not, @@ -92,6 +108,7 @@ impl OperatorMode { Expr::Like(like) } + /// Creates an [`Expr::BinaryExpr`] of "`left` = `right`" or "`left` != `right`". fn expr_matches_literal(&self, left: Box, right: Box) -> Expr { let op = if self.not { Operator::NotEq @@ -118,7 +135,7 @@ fn collect_concat_to_like_string(parts: &[Hir]) -> Option { Some(s) } -/// returns a str represented by `Literal` if it contains a valid utf8 +/// Returns a str represented by `Literal` if it contains a valid utf8 /// sequence and is safe for like (has no '%' and '_') fn like_str_from_literal(l: &Literal) -> Option<&str> { // if not utf8, no good @@ -131,7 +148,7 @@ fn like_str_from_literal(l: &Literal) -> Option<&str> { } } -/// returns a str represented by `Literal` if it contains a valid utf8 +/// Returns a str represented by `Literal` if it contains a valid utf8 fn str_from_literal(l: &Literal) -> Option<&str> { // if not utf8, no good let s = std::str::from_utf8(&l.0).ok()?; @@ -143,7 +160,7 @@ fn is_safe_for_like(c: char) -> bool { (c != '%') && (c != '_') } -/// returns true if the elements in a `Concat` pattern are: +/// Returns true if the elements in a `Concat` pattern are: /// - `[Look::Start, Look::End]` /// - `[Look::Start, Literal(_), Look::End]` fn is_anchored_literal(v: &[Hir]) -> bool { @@ -157,10 +174,9 @@ fn is_anchored_literal(v: &[Hir]) -> bool { v.last().expect("length checked"), ); if !matches!(first_last, - (s, e) if s.kind() == &HirKind::Look(Look::Start) + (s, e) if s.kind() == &HirKind::Look(Look::Start) && e.kind() == &HirKind::Look(Look::End) - ) - { + ) { return false; } @@ -170,7 +186,7 @@ fn is_anchored_literal(v: &[Hir]) -> bool { .all(|h| matches!(h.kind(), HirKind::Literal(_))) } -/// returns true if the elements in a `Concat` pattern are: +/// Returns true if the elements in a `Concat` pattern are: /// - `[Look::Start, Capture(Alternation(Literals...)), Look::End]` fn is_anchored_capture(v: &[Hir]) -> bool { if v.len() != 3 @@ -197,7 +213,33 @@ fn is_anchored_capture(v: &[Hir]) -> bool { true } -/// extracts a string literal expression assuming that [`is_anchored_literal`] +/// Returns the `LIKE` pattern if the `Concat` pattern is partial anchored: +/// - `[Look::Start, Literal(_)]` +/// - `[Literal(_), Look::End]` +/// Full anchored patterns are handled by [`anchored_literal_to_expr`]. +fn partial_anchored_literal_to_like(v: &[Hir]) -> Option { + if v.len() != 2 { + return None; + } + + let (lit, match_begin) = match (&v[0].kind(), &v[1].kind()) { + (HirKind::Look(Look::Start), HirKind::Literal(l)) => { + (like_str_from_literal(l)?, true) + } + (HirKind::Literal(l), HirKind::Look(Look::End)) => { + (like_str_from_literal(l)?, false) + } + _ => return None, + }; + + if match_begin { + Some(format!("{}%", lit)) + } else { + Some(format!("%{}", lit)) + } +} + +/// Extracts a string literal expression assuming that [`is_anchored_literal`] /// returned true. fn anchored_literal_to_expr(v: &[Hir]) -> Option { match v.len() { @@ -246,6 +288,7 @@ fn anchored_alternation_to_exprs(v: &[Hir]) -> Option> { None } +/// Tries to lower (transform) a simple regex pattern to a LIKE expression. fn lower_simple(mode: &OperatorMode, left: &Expr, hir: &Hir) -> Option { match hir.kind() { HirKind::Empty => { @@ -265,7 +308,9 @@ fn lower_simple(mode: &OperatorMode, left: &Expr, hir: &Hir) -> Option { .map(|right| left.clone().in_list(right, mode.not)); } HirKind::Concat(inner) => { - if let Some(pattern) = collect_concat_to_like_string(inner) { + if let Some(pattern) = partial_anchored_literal_to_like(inner) + .or(collect_concat_to_like_string(inner)) + { return Some(mode.expr(Box::new(left.clone()), pattern)); } } @@ -274,6 +319,9 @@ fn lower_simple(mode: &OperatorMode, left: &Expr, hir: &Hir) -> Option { None } +/// Calls [`lower_simple`] for each alternative and combine the results with `or` or `and` +/// based on [`OperatorMode`]. Any fail attempt to lower an alternative will makes this +/// function to return `None`. fn lower_alt(mode: &OperatorMode, left: &Expr, alts: &[Hir]) -> Option { let mut accu: Option = None; From 4109f581ce9bca956e01f13ff16b30d59720e96b Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Wed, 19 Jun 2024 11:36:16 +0800 Subject: [PATCH 24/54] Push down filter for Unnest plan (#10974) * add unnset Signed-off-by: jayzhan211 * improve doc and tset Signed-off-by: jayzhan211 * fix test Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/optimizer/src/push_down_filter.rs | 156 ++++++++++++------ .../test_files/push_down_filter.slt | 51 ++++++ 2 files changed, 158 insertions(+), 49 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/push_down_filter.slt diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 2cc50c7f82b6..8dc71b70e22b 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -35,7 +35,7 @@ use datafusion_expr::logical_plan::{ use datafusion_expr::utils::{conjunction, split_conjunction, split_conjunction_owned}; use datafusion_expr::{ and, build_join_schema, or, BinaryExpr, Expr, Filter, LogicalPlanBuilder, Operator, - TableProviderFilterPushDown, + Projection, TableProviderFilterPushDown, }; use crate::optimizer::ApplyOrder; @@ -691,58 +691,46 @@ impl OptimizerRule for PushDownFilter { insert_below(LogicalPlan::SubqueryAlias(subquery_alias), new_filter) } LogicalPlan::Projection(projection) => { - // A projection is filter-commutable if it do not contain volatile predicates or contain volatile - // predicates that are not used in the filter. However, we should re-writes all predicate expressions. - // collect projection. - let (volatile_map, non_volatile_map): (HashMap<_, _>, HashMap<_, _>) = - projection - .schema - .iter() - .zip(projection.expr.iter()) - .map(|((qualifier, field), expr)| { - // strip alias, as they should not be part of filters - let expr = expr.clone().unalias(); - - (qualified_name(qualifier, field.name()), expr) - }) - .partition(|(_, value)| value.is_volatile().unwrap_or(true)); - - let mut push_predicates = vec![]; - let mut keep_predicates = vec![]; - for expr in split_conjunction_owned(filter.predicate.clone()) { - if contain(&expr, &volatile_map) { - keep_predicates.push(expr); - } else { - push_predicates.push(expr); + let (new_projection, keep_predicate) = + rewrite_projection(filter.predicate.clone(), projection)?; + if new_projection.transformed { + match keep_predicate { + None => Ok(new_projection), + Some(keep_predicate) => new_projection.map_data(|child_plan| { + Filter::try_new(keep_predicate, Arc::new(child_plan)) + .map(LogicalPlan::Filter) + }), } + } else { + filter.input = Arc::new(new_projection.data); + Ok(Transformed::no(LogicalPlan::Filter(filter))) } - - match conjunction(push_predicates) { - Some(expr) => { - // re-write all filters based on this projection - // E.g. in `Filter: b\n Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1" - let new_filter = LogicalPlan::Filter(Filter::try_new( - replace_cols_by_name(expr, &non_volatile_map)?, - Arc::clone(&projection.input), - )?); - - match conjunction(keep_predicates) { - None => insert_below( - LogicalPlan::Projection(projection), - new_filter, - ), - Some(keep_predicate) => insert_below( - LogicalPlan::Projection(projection), - new_filter, - )? - .map_data(|child_plan| { - Filter::try_new(keep_predicate, Arc::new(child_plan)) - .map(LogicalPlan::Filter) - }), + } + LogicalPlan::Unnest(mut unnest) => { + // Unnest is built above Projection, so we only take Projection into consideration + match unwrap_arc(unnest.input) { + LogicalPlan::Projection(projection) => { + let (new_projection, keep_predicate) = + rewrite_projection(filter.predicate.clone(), projection)?; + unnest.input = Arc::new(new_projection.data); + + if new_projection.transformed { + match keep_predicate { + None => Ok(Transformed::yes(LogicalPlan::Unnest(unnest))), + Some(keep_predicate) => Ok(Transformed::yes( + LogicalPlan::Filter(Filter::try_new( + keep_predicate, + Arc::new(LogicalPlan::Unnest(unnest)), + )?), + )), + } + } else { + filter.input = Arc::new(LogicalPlan::Unnest(unnest)); + Ok(Transformed::no(LogicalPlan::Filter(filter))) } } - None => { - filter.input = Arc::new(LogicalPlan::Projection(projection)); + child => { + filter.input = Arc::new(child); Ok(Transformed::no(LogicalPlan::Filter(filter))) } } @@ -951,6 +939,76 @@ impl OptimizerRule for PushDownFilter { } } +/// Attempts to push `predicate` into a `FilterExec` below `projection +/// +/// # Returns +/// (plan, remaining_predicate) +/// +/// `plan` is a LogicalPlan for `projection` with possibly a new FilterExec below it. +/// `remaining_predicate` is any part of the predicate that could not be pushed down +/// +/// # Example +/// +/// Pushing a predicate like `foo=5 AND bar=6` with an input plan like this: +/// +/// ```text +/// Projection(foo, c+d as bar) +/// ``` +/// +/// Might result in returning `remaining_predicate` of `bar=6` and a plan like +/// +/// ```text +/// Projection(foo, c+d as bar) +/// Filter(foo=5) +/// ... +/// ``` +fn rewrite_projection( + predicate: Expr, + projection: Projection, +) -> Result<(Transformed, Option)> { + // A projection is filter-commutable if it do not contain volatile predicates or contain volatile + // predicates that are not used in the filter. However, we should re-writes all predicate expressions. + // collect projection. + let (volatile_map, non_volatile_map): (HashMap<_, _>, HashMap<_, _>) = projection + .schema + .iter() + .zip(projection.expr.iter()) + .map(|((qualifier, field), expr)| { + // strip alias, as they should not be part of filters + let expr = expr.clone().unalias(); + + (qualified_name(qualifier, field.name()), expr) + }) + .partition(|(_, value)| value.is_volatile().unwrap_or(true)); + + let mut push_predicates = vec![]; + let mut keep_predicates = vec![]; + for expr in split_conjunction_owned(predicate) { + if contain(&expr, &volatile_map) { + keep_predicates.push(expr); + } else { + push_predicates.push(expr); + } + } + + match conjunction(push_predicates) { + Some(expr) => { + // re-write all filters based on this projection + // E.g. in `Filter: b\n Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1" + let new_filter = LogicalPlan::Filter(Filter::try_new( + replace_cols_by_name(expr, &non_volatile_map)?, + Arc::clone(&projection.input), + )?); + + Ok(( + insert_below(LogicalPlan::Projection(projection), new_filter)?, + conjunction(keep_predicates), + )) + } + None => Ok((Transformed::no(LogicalPlan::Projection(projection)), None)), + } +} + /// Creates a new LogicalPlan::Filter node. pub fn make_filter(predicate: Expr, input: Arc) -> Result { Filter::try_new(predicate, input).map(LogicalPlan::Filter) diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt new file mode 100644 index 000000000000..1d72e11f4352 --- /dev/null +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -0,0 +1,51 @@ +# 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. + +# Test push down filter + +statement ok +set datafusion.explain.logical_plan_only = true; + +statement ok +CREATE TABLE IF NOT EXISTS v AS VALUES(1,[1,2,3]),(2,[3,4,5]); + +query I +select uc2 from (select unnest(column2) as uc2, column1 from v) where column1 = 2; +---- +3 +4 +5 + +# test push down filter for unnest with filter on non-unnest column +# filter plan is pushed down into projection plan +query TT +explain select uc2 from (select unnest(column2) as uc2, column1 from v) where column1 = 2; +---- +logical_plan +01)Projection: unnest(v.column2) AS uc2 +02)--Unnest: lists[unnest(v.column2)] structs[] +03)----Projection: v.column2 AS unnest(v.column2), v.column1 +04)------Filter: v.column1 = Int64(2) +05)--------TableScan: v projection=[column1, column2] + +# TODO: fix the query +query error DataFusion error: External error: Arrow error: Invalid argument error: Invalid comparison operation: List\(Field \{ name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: \{\} \}\) > Int64 +select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; + +# test push down filter for unnest with filter on unnest column +# query TT +# explain select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; From 268f648db9175cb8a24c46c6e40cb5bc03995b2b Mon Sep 17 00:00:00 2001 From: tmi Date: Wed, 19 Jun 2024 15:46:24 +0200 Subject: [PATCH 25/54] Minor: add parquet page stats for float{16, 32, 64} (#10982) --- .../physical_plan/parquet/statistics.rs | 100 ++++++++++++++++-- .../core/tests/parquet/arrow_statistics.rs | 88 +++++++++++++++ datafusion/core/tests/parquet/mod.rs | 15 +++ 3 files changed, 195 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index 6ad78a82b9bf..2ca47de990ec 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -33,8 +33,9 @@ use arrow_array::{ use arrow_schema::{Field, FieldRef, Schema, TimeUnit}; use datafusion_common::{internal_datafusion_err, internal_err, plan_err, Result}; use half::f16; +use parquet::data_type::FixedLenByteArray; use parquet::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex, RowGroupMetaData}; -use parquet::file::page_index::index::Index; +use parquet::file::page_index::index::{Index, PageIndex}; use parquet::file::statistics::Statistics as ParquetStatistics; use parquet::schema::types::SchemaDescriptor; use paste::paste; @@ -495,7 +496,7 @@ macro_rules! get_statistics { } macro_rules! make_data_page_stats_iterator { - ($iterator_type: ident, $func: ident, $index_type: path, $stat_value_type: ty) => { + ($iterator_type: ident, $func: expr, $index_type: path, $stat_value_type: ty) => { struct $iterator_type<'a, I> where I: Iterator, @@ -526,7 +527,7 @@ macro_rules! make_data_page_stats_iterator { native_index .indexes .iter() - .map(|x| x.$func) + .map(|x| $func(x)) .collect::>(), ), // No matching `Index` found; @@ -548,11 +549,66 @@ macro_rules! make_data_page_stats_iterator { }; } -make_data_page_stats_iterator!(MinInt32DataPageStatsIterator, min, Index::INT32, i32); -make_data_page_stats_iterator!(MaxInt32DataPageStatsIterator, max, Index::INT32, i32); -make_data_page_stats_iterator!(MinInt64DataPageStatsIterator, min, Index::INT64, i64); -make_data_page_stats_iterator!(MaxInt64DataPageStatsIterator, max, Index::INT64, i64); - +make_data_page_stats_iterator!( + MinInt32DataPageStatsIterator, + |x: &PageIndex| { x.min }, + Index::INT32, + i32 +); +make_data_page_stats_iterator!( + MaxInt32DataPageStatsIterator, + |x: &PageIndex| { x.max }, + Index::INT32, + i32 +); +make_data_page_stats_iterator!( + MinInt64DataPageStatsIterator, + |x: &PageIndex| { x.min }, + Index::INT64, + i64 +); +make_data_page_stats_iterator!( + MaxInt64DataPageStatsIterator, + |x: &PageIndex| { x.max }, + Index::INT64, + i64 +); +make_data_page_stats_iterator!( + MinFloat16DataPageStatsIterator, + |x: &PageIndex| { x.min.clone() }, + Index::FIXED_LEN_BYTE_ARRAY, + FixedLenByteArray +); +make_data_page_stats_iterator!( + MaxFloat16DataPageStatsIterator, + |x: &PageIndex| { x.max.clone() }, + Index::FIXED_LEN_BYTE_ARRAY, + FixedLenByteArray +); +make_data_page_stats_iterator!( + MinFloat32DataPageStatsIterator, + |x: &PageIndex| { x.min }, + Index::FLOAT, + f32 +); +make_data_page_stats_iterator!( + MaxFloat32DataPageStatsIterator, + |x: &PageIndex| { x.max }, + Index::FLOAT, + f32 +); +make_data_page_stats_iterator!( + MinFloat64DataPageStatsIterator, + |x: &PageIndex| { x.min }, + Index::DOUBLE, + f64 +); +make_data_page_stats_iterator!( + MaxFloat64DataPageStatsIterator, + |x: &PageIndex| { x.max }, + Index::DOUBLE, + f64 +); macro_rules! get_data_page_statistics { ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { paste! { @@ -581,6 +637,19 @@ macro_rules! get_data_page_statistics { )), Some(DataType::Int32) => Ok(Arc::new(Int32Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten()))), Some(DataType::Int64) => Ok(Arc::new(Int64Array::from_iter([<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator).flatten()))), + Some(DataType::Float16) => Ok(Arc::new( + Float16Array::from_iter( + [<$stat_type_prefix Float16DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| Some(from_bytes_to_f16(x.data()))) + }) + }) + .flatten() + ) + )), + Some(DataType::Float32) => Ok(Arc::new(Float32Array::from_iter([<$stat_type_prefix Float32DataPageStatsIterator>]::new($iterator).flatten()))), + Some(DataType::Float64) => Ok(Arc::new(Float64Array::from_iter([<$stat_type_prefix Float64DataPageStatsIterator>]::new($iterator).flatten()))), _ => unimplemented!() } } @@ -677,6 +746,21 @@ where .iter() .map(|x| x.null_count.map(|x| x as u64)) .collect::>(), + Index::FLOAT(native_index) => native_index + .indexes + .iter() + .map(|x| x.null_count.map(|x| x as u64)) + .collect::>(), + Index::DOUBLE(native_index) => native_index + .indexes + .iter() + .map(|x| x.null_count.map(|x| x as u64)) + .collect::>(), + Index::FIXED_LEN_BYTE_ARRAY(native_index) => native_index + .indexes + .iter() + .map(|x| x.null_count.map(|x| x as u64)) + .collect::>(), _ => unimplemented!(), }); diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index 4c68a57333e5..bdae9f47867e 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -614,6 +614,94 @@ async fn test_int_8() { .run(); } +#[tokio::test] +async fn test_float_16() { + // This creates a parquet files of 1 column named f + let reader = TestReader { + scenario: Scenario::Float16, + row_per_group: 5, + } + .build() + .await; + + Test { + reader: &reader, + // mins are [-5, -4, 0, 5] + expected_min: Arc::new(Float16Array::from(vec![ + f16::from_f32(-5.), + f16::from_f32(-4.), + f16::from_f32(-0.), + f16::from_f32(5.), + ])), + // maxes are [-1, 0, 4, 9] + expected_max: Arc::new(Float16Array::from(vec![ + f16::from_f32(-1.), + f16::from_f32(0.), + f16::from_f32(4.), + f16::from_f32(9.), + ])), + // nulls are [0, 0, 0, 0] + expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), + // row counts are [5, 5, 5, 5] + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), + column_name: "f", + check: Check::Both, + } + .run(); +} + +#[tokio::test] +async fn test_float_32() { + // This creates a parquet files of 1 column named f + let reader = TestReader { + scenario: Scenario::Float32, + row_per_group: 5, + } + .build() + .await; + + Test { + reader: &reader, + // mins are [-5, -4, 0, 5] + expected_min: Arc::new(Float32Array::from(vec![-5., -4., -0., 5.0])), + // maxes are [-1, 0, 4, 9] + expected_max: Arc::new(Float32Array::from(vec![-1., 0., 4., 9.])), + // nulls are [0, 0, 0, 0] + expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), + // row counts are [5, 5, 5, 5] + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), + column_name: "f", + check: Check::Both, + } + .run(); +} + +#[tokio::test] +async fn test_float_64() { + // This creates a parquet files of 1 column named f + let reader = TestReader { + scenario: Scenario::Float64, + row_per_group: 5, + } + .build() + .await; + + Test { + reader: &reader, + // mins are [-5, -4, 0, 5] + expected_min: Arc::new(Float64Array::from(vec![-5., -4., -0., 5.0])), + // maxes are [-1, 0, 4, 9] + expected_max: Arc::new(Float64Array::from(vec![-1., 0., 4., 9.])), + // nulls are [0, 0, 0, 0] + expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0]), + // row counts are [5, 5, 5, 5] + expected_row_counts: Some(UInt64Array::from(vec![5, 5, 5, 5])), + column_name: "f", + check: Check::Both, + } + .run(); +} + // timestamp #[tokio::test] async fn test_timestamp() { diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 0434a271c32e..1b68a4aa4eb3 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -90,6 +90,7 @@ enum Scenario { /// -MIN, -100, -1, 0, 1, 100, MAX NumericLimits, Float16, + Float32, Float64, Decimal, Decimal256, @@ -586,6 +587,12 @@ fn make_f64_batch(v: Vec) -> RecordBatch { RecordBatch::try_new(schema, vec![array.clone()]).unwrap() } +fn make_f32_batch(v: Vec) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Float32, true)])); + let array = Arc::new(Float32Array::from(v)) as ArrayRef; + RecordBatch::try_new(schema, vec![array.clone()]).unwrap() +} + fn make_f16_batch(v: Vec) -> RecordBatch { let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Float16, true)])); let array = Arc::new(Float16Array::from(v)) as ArrayRef; @@ -1003,6 +1010,14 @@ fn create_data_batch(scenario: Scenario) -> Vec { ), ] } + Scenario::Float32 => { + vec![ + make_f32_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), + make_f32_batch(vec![-4.0, -3.0, -2.0, -1.0, 0.0]), + make_f32_batch(vec![0.0, 1.0, 2.0, 3.0, 4.0]), + make_f32_batch(vec![5.0, 6.0, 7.0, 8.0, 9.0]), + ] + } Scenario::Float64 => { vec![ make_f64_batch(vec![-5.0, -4.0, -3.0, -2.0, -1.0]), From ea0ba99d94172780ea6e84010362bc1967d39748 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 19 Jun 2024 21:58:45 +0800 Subject: [PATCH 26/54] Fix `file_stream_provider` example compilation failure on windows (#10975) * Fix file_stream_provider on windows * Add .await & fmt --- .../examples/file_stream_provider.rs | 326 +++++++++--------- 1 file changed, 171 insertions(+), 155 deletions(-) diff --git a/datafusion-examples/examples/file_stream_provider.rs b/datafusion-examples/examples/file_stream_provider.rs index 4e79f9afc2ca..b8549bd6b6e6 100644 --- a/datafusion-examples/examples/file_stream_provider.rs +++ b/datafusion-examples/examples/file_stream_provider.rs @@ -15,172 +15,188 @@ // specific language governing permissions and limitations // under the License. -use datafusion::assert_batches_eq; -use datafusion_common::instant::Instant; -use std::fs::{File, OpenOptions}; -use std::io::Write; -use std::path::PathBuf; -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::Arc; -use std::thread; -use std::time::Duration; - -use arrow::datatypes::{DataType, Field, Schema}; -use arrow_schema::SchemaRef; -use futures::StreamExt; -use nix::sys::stat; -use nix::unistd; -use tempfile::TempDir; -use tokio::task::JoinSet; - -use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; -use datafusion::datasource::TableProvider; -use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_common::{exec_err, Result}; -use datafusion_expr::Expr; - -// Number of lines written to FIFO -const TEST_BATCH_SIZE: usize = 5; -const TEST_DATA_SIZE: usize = 5; - -/// Makes a TableProvider for a fifo file using `StreamTable` with the `StreamProvider` trait -fn fifo_table( - schema: SchemaRef, - path: impl Into, - sort: Vec>, -) -> Arc { - let source = FileStreamProvider::new_file(schema, path.into()) - .with_batch_size(TEST_BATCH_SIZE) - .with_header(true); - let config = StreamConfig::new(Arc::new(source)).with_order(sort); - Arc::new(StreamTable::new(Arc::new(config))) -} +#[cfg(not(target_os = "windows"))] +mod non_windows { + use datafusion::assert_batches_eq; + use datafusion_common::instant::Instant; + use std::fs::{File, OpenOptions}; + use std::io::Write; + use std::path::PathBuf; + use std::sync::atomic::{AtomicBool, Ordering}; + use std::sync::Arc; + use std::thread; + use std::time::Duration; + + use arrow::datatypes::{DataType, Field, Schema}; + use arrow_schema::SchemaRef; + use futures::StreamExt; + use nix::sys::stat; + use nix::unistd; + use tempfile::TempDir; + use tokio::task::JoinSet; + + use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; + use datafusion::datasource::TableProvider; + use datafusion::prelude::{SessionConfig, SessionContext}; + use datafusion_common::{exec_err, Result}; + use datafusion_expr::Expr; + + // Number of lines written to FIFO + const TEST_BATCH_SIZE: usize = 5; + const TEST_DATA_SIZE: usize = 5; + + /// Makes a TableProvider for a fifo file using `StreamTable` with the `StreamProvider` trait + fn fifo_table( + schema: SchemaRef, + path: impl Into, + sort: Vec>, + ) -> Arc { + let source = FileStreamProvider::new_file(schema, path.into()) + .with_batch_size(TEST_BATCH_SIZE) + .with_header(true); + let config = StreamConfig::new(Arc::new(source)).with_order(sort); + Arc::new(StreamTable::new(Arc::new(config))) + } -fn create_fifo_file(tmp_dir: &TempDir, file_name: &str) -> Result { - let file_path = tmp_dir.path().join(file_name); - // Simulate an infinite environment via a FIFO file - if let Err(e) = unistd::mkfifo(&file_path, stat::Mode::S_IRWXU) { - exec_err!("{}", e) - } else { - Ok(file_path) + fn create_fifo_file(tmp_dir: &TempDir, file_name: &str) -> Result { + let file_path = tmp_dir.path().join(file_name); + // Simulate an infinite environment via a FIFO file + if let Err(e) = unistd::mkfifo(&file_path, stat::Mode::S_IRWXU) { + exec_err!("{}", e) + } else { + Ok(file_path) + } } -} -fn write_to_fifo( - mut file: &File, - line: &str, - ref_time: Instant, - broken_pipe_timeout: Duration, -) -> Result<()> { - // We need to handle broken pipe error until the reader is ready. This - // is why we use a timeout to limit the wait duration for the reader. - // If the error is different than broken pipe, we fail immediately. - while let Err(e) = file.write_all(line.as_bytes()) { - if e.raw_os_error().unwrap() == 32 { - let interval = Instant::now().duration_since(ref_time); - if interval < broken_pipe_timeout { - thread::sleep(Duration::from_millis(100)); - continue; + fn write_to_fifo( + mut file: &File, + line: &str, + ref_time: Instant, + broken_pipe_timeout: Duration, + ) -> Result<()> { + // We need to handle broken pipe error until the reader is ready. This + // is why we use a timeout to limit the wait duration for the reader. + // If the error is different than broken pipe, we fail immediately. + while let Err(e) = file.write_all(line.as_bytes()) { + if e.raw_os_error().unwrap() == 32 { + let interval = Instant::now().duration_since(ref_time); + if interval < broken_pipe_timeout { + thread::sleep(Duration::from_millis(100)); + continue; + } } + return exec_err!("{}", e); } - return exec_err!("{}", e); + Ok(()) } - Ok(()) -} -fn create_writing_thread( - file_path: PathBuf, - maybe_header: Option, - lines: Vec, - waiting_lock: Arc, - wait_until: usize, - tasks: &mut JoinSet<()>, -) { - // Timeout for a long period of BrokenPipe error - let broken_pipe_timeout = Duration::from_secs(10); - let sa = file_path.clone(); - // Spawn a new thread to write to the FIFO file - #[allow(clippy::disallowed_methods)] // spawn allowed only in tests - tasks.spawn_blocking(move || { - let file = OpenOptions::new().write(true).open(sa).unwrap(); - // Reference time to use when deciding to fail the test - let execution_start = Instant::now(); - if let Some(header) = maybe_header { - write_to_fifo(&file, &header, execution_start, broken_pipe_timeout).unwrap(); - } - for (cnt, line) in lines.iter().enumerate() { - while waiting_lock.load(Ordering::SeqCst) && cnt > wait_until { - thread::sleep(Duration::from_millis(50)); + fn create_writing_thread( + file_path: PathBuf, + maybe_header: Option, + lines: Vec, + waiting_lock: Arc, + wait_until: usize, + tasks: &mut JoinSet<()>, + ) { + // Timeout for a long period of BrokenPipe error + let broken_pipe_timeout = Duration::from_secs(10); + let sa = file_path.clone(); + // Spawn a new thread to write to the FIFO file + #[allow(clippy::disallowed_methods)] // spawn allowed only in tests + tasks.spawn_blocking(move || { + let file = OpenOptions::new().write(true).open(sa).unwrap(); + // Reference time to use when deciding to fail the test + let execution_start = Instant::now(); + if let Some(header) = maybe_header { + write_to_fifo(&file, &header, execution_start, broken_pipe_timeout) + .unwrap(); + } + for (cnt, line) in lines.iter().enumerate() { + while waiting_lock.load(Ordering::SeqCst) && cnt > wait_until { + thread::sleep(Duration::from_millis(50)); + } + write_to_fifo(&file, line, execution_start, broken_pipe_timeout).unwrap(); } - write_to_fifo(&file, line, execution_start, broken_pipe_timeout).unwrap(); + drop(file); + }); + } + + /// This example demonstrates a scanning against an Arrow data source (JSON) and + /// fetching results + pub async fn main() -> Result<()> { + // Create session context + let config = SessionConfig::new() + .with_batch_size(TEST_BATCH_SIZE) + .with_collect_statistics(false) + .with_target_partitions(1); + let ctx = SessionContext::new_with_config(config); + let tmp_dir = TempDir::new()?; + let fifo_path = create_fifo_file(&tmp_dir, "fifo_unbounded.csv")?; + + let mut tasks: JoinSet<()> = JoinSet::new(); + let waiting = Arc::new(AtomicBool::new(true)); + + let data_iter = 0..TEST_DATA_SIZE; + let lines = data_iter + .map(|i| format!("{},{}\n", i, i + 1)) + .collect::>(); + + create_writing_thread( + fifo_path.clone(), + Some("a1,a2\n".to_owned()), + lines.clone(), + waiting.clone(), + TEST_DATA_SIZE, + &mut tasks, + ); + + // Create schema + let schema = Arc::new(Schema::new(vec![ + Field::new("a1", DataType::UInt32, false), + Field::new("a2", DataType::UInt32, false), + ])); + + // Specify the ordering: + let order = vec![vec![datafusion_expr::col("a1").sort(true, false)]]; + + let provider = fifo_table(schema.clone(), fifo_path, order.clone()); + ctx.register_table("fifo", provider)?; + + let df = ctx.sql("SELECT * FROM fifo").await.unwrap(); + let mut stream = df.execute_stream().await.unwrap(); + + let mut batches = Vec::new(); + if let Some(Ok(batch)) = stream.next().await { + batches.push(batch) } - drop(file); - }); + + let expected = vec![ + "+----+----+", + "| a1 | a2 |", + "+----+----+", + "| 0 | 1 |", + "| 1 | 2 |", + "| 2 | 3 |", + "| 3 | 4 |", + "| 4 | 5 |", + "+----+----+", + ]; + + assert_batches_eq!(&expected, &batches); + + Ok(()) + } } -/// This example demonstrates a scanning against an Arrow data source (JSON) and -/// fetching results #[tokio::main] -async fn main() -> Result<()> { - // Create session context - let config = SessionConfig::new() - .with_batch_size(TEST_BATCH_SIZE) - .with_collect_statistics(false) - .with_target_partitions(1); - let ctx = SessionContext::new_with_config(config); - let tmp_dir = TempDir::new()?; - let fifo_path = create_fifo_file(&tmp_dir, "fifo_unbounded.csv")?; - - let mut tasks: JoinSet<()> = JoinSet::new(); - let waiting = Arc::new(AtomicBool::new(true)); - - let data_iter = 0..TEST_DATA_SIZE; - let lines = data_iter - .map(|i| format!("{},{}\n", i, i + 1)) - .collect::>(); - - create_writing_thread( - fifo_path.clone(), - Some("a1,a2\n".to_owned()), - lines.clone(), - waiting.clone(), - TEST_DATA_SIZE, - &mut tasks, - ); - - // Create schema - let schema = Arc::new(Schema::new(vec![ - Field::new("a1", DataType::UInt32, false), - Field::new("a2", DataType::UInt32, false), - ])); - - // Specify the ordering: - let order = vec![vec![datafusion_expr::col("a1").sort(true, false)]]; - - let provider = fifo_table(schema.clone(), fifo_path, order.clone()); - ctx.register_table("fifo", provider)?; - - let df = ctx.sql("SELECT * FROM fifo").await.unwrap(); - let mut stream = df.execute_stream().await.unwrap(); - - let mut batches = Vec::new(); - if let Some(Ok(batch)) = stream.next().await { - batches.push(batch) +async fn main() -> datafusion_common::Result<()> { + #[cfg(target_os = "windows")] + { + println!("file_stream_provider example does not work on windows"); + Ok(()) + } + #[cfg(not(target_os = "windows"))] + { + non_windows::main().await } - - let expected = vec![ - "+----+----+", - "| a1 | a2 |", - "+----+----+", - "| 0 | 1 |", - "| 1 | 2 |", - "| 2 | 3 |", - "| 3 | 4 |", - "| 4 | 5 |", - "+----+----+", - ]; - - assert_batches_eq!(&expected, &batches); - - Ok(()) } From c6b2efccf6238cc87f2414efb28ae3b263ed27af Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 19 Jun 2024 10:13:27 -0400 Subject: [PATCH 27/54] Stop copying LogicalPlan and Exprs in `CommonSubexprEliminate` (2-3% planning speed improvement) (#10835) * Stop copying LogicalPlan and Exprs in `CommonSubexprEliminate` * thread transformed * Update unary to report transformed correctly * Preserve through window transforms * track aggregate * Avoid re-computing Aggregate schema * Update datafusion/optimizer/src/common_subexpr_eliminate.rs * Avoid unecessary setting transform flat * Cleanup unaliasing --- datafusion/common/src/tree_node.rs | 5 + datafusion/expr/src/logical_plan/plan.rs | 64 +- .../optimizer/src/common_subexpr_eliminate.rs | 594 ++++++++++++------ 3 files changed, 439 insertions(+), 224 deletions(-) diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index d0dd24621d3e..276a1cc4c59c 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -615,6 +615,11 @@ impl Transformed { } } + /// Create a `Transformed` with `transformed and [`TreeNodeRecursion::Continue`]. + pub fn new_transformed(data: T, transformed: bool) -> Self { + Self::new(data, transformed, TreeNodeRecursion::Continue) + } + /// Wrapper for transformed data with [`TreeNodeRecursion::Continue`] statement. pub fn yes(data: T) -> Self { Self::new(data, true, TreeNodeRecursion::Continue) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 02378ab3fc1b..85958223ac97 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -870,37 +870,7 @@ impl LogicalPlan { LogicalPlan::Filter { .. } => { assert_eq!(1, expr.len()); let predicate = expr.pop().unwrap(); - - // filter predicates should not contain aliased expressions so we remove any aliases - // before this logic was added we would have aliases within filters such as for - // benchmark q6: - // - // lineitem.l_shipdate >= Date32(\"8766\") - // AND lineitem.l_shipdate < Date32(\"9131\") - // AND CAST(lineitem.l_discount AS Decimal128(30, 15)) AS lineitem.l_discount >= - // Decimal128(Some(49999999999999),30,15) - // AND CAST(lineitem.l_discount AS Decimal128(30, 15)) AS lineitem.l_discount <= - // Decimal128(Some(69999999999999),30,15) - // AND lineitem.l_quantity < Decimal128(Some(2400),15,2) - - let predicate = predicate - .transform_down(|expr| { - match expr { - Expr::Exists { .. } - | Expr::ScalarSubquery(_) - | Expr::InSubquery(_) => { - // subqueries could contain aliases so we don't recurse into those - Ok(Transformed::new(expr, false, TreeNodeRecursion::Jump)) - } - Expr::Alias(_) => Ok(Transformed::new( - expr.unalias(), - true, - TreeNodeRecursion::Jump, - )), - _ => Ok(Transformed::no(expr)), - } - }) - .data()?; + let predicate = Filter::remove_aliases(predicate)?.data; Filter::try_new(predicate, Arc::new(inputs.swap_remove(0))) .map(LogicalPlan::Filter) @@ -2230,6 +2200,38 @@ impl Filter { } false } + + /// Remove aliases from a predicate for use in a `Filter` + /// + /// filter predicates should not contain aliased expressions so we remove + /// any aliases. + /// + /// before this logic was added we would have aliases within filters such as + /// for benchmark q6: + /// + /// ```sql + /// lineitem.l_shipdate >= Date32(\"8766\") + /// AND lineitem.l_shipdate < Date32(\"9131\") + /// AND CAST(lineitem.l_discount AS Decimal128(30, 15)) AS lineitem.l_discount >= + /// Decimal128(Some(49999999999999),30,15) + /// AND CAST(lineitem.l_discount AS Decimal128(30, 15)) AS lineitem.l_discount <= + /// Decimal128(Some(69999999999999),30,15) + /// AND lineitem.l_quantity < Decimal128(Some(2400),15,2) + /// ``` + pub fn remove_aliases(predicate: Expr) -> Result> { + predicate.transform_down(|expr| { + match expr { + Expr::Exists { .. } | Expr::ScalarSubquery(_) | Expr::InSubquery(_) => { + // subqueries could contain aliases so we don't recurse into those + Ok(Transformed::new(expr, false, TreeNodeRecursion::Jump)) + } + Expr::Alias(Alias { expr, .. }) => { + Ok(Transformed::new(*expr, true, TreeNodeRecursion::Jump)) + } + _ => Ok(Transformed::no(expr)), + } + }) + } } /// Window its input based on a set of window spec and window function (e.g. SUM or RANK) diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index e150a957bfcf..7f4093ba110e 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -20,16 +20,22 @@ use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; -use crate::{utils, OptimizerConfig, OptimizerRule}; +use crate::{OptimizerConfig, OptimizerRule}; +use crate::optimizer::ApplyOrder; +use crate::utils::NamePreserver; use datafusion_common::alias::AliasGenerator; use datafusion_common::tree_node::{ - Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter, - TreeNodeVisitor, + Transformed, TreeNode, TreeNodeRecursion, TreeNodeRewriter, TreeNodeVisitor, +}; +use datafusion_common::{ + internal_datafusion_err, internal_err, qualified_name, Column, DFSchema, Result, }; -use datafusion_common::{qualified_name, Column, DFSchema, DataFusionError, Result}; use datafusion_expr::expr::Alias; -use datafusion_expr::logical_plan::{Aggregate, LogicalPlan, Projection, Window}; +use datafusion_expr::logical_plan::tree_node::unwrap_arc; +use datafusion_expr::logical_plan::{ + Aggregate, Filter, LogicalPlan, Projection, Sort, Window, +}; use datafusion_expr::{col, Expr, ExprSchemable}; use indexmap::IndexMap; @@ -123,32 +129,39 @@ impl CommonSubexprEliminate { /// Returns the rewritten expressions fn rewrite_exprs_list( &self, - exprs_list: &[&[Expr]], + exprs_list: Vec>, arrays_list: &[&[IdArray]], expr_stats: &ExprStats, common_exprs: &mut CommonExprs, alias_generator: &AliasGenerator, - ) -> Result>> { + ) -> Result>>> { + let mut transformed = false; exprs_list - .iter() + .into_iter() .zip(arrays_list.iter()) .map(|(exprs, arrays)| { exprs - .iter() - .cloned() + .into_iter() .zip(arrays.iter()) .map(|(expr, id_array)| { - replace_common_expr( + let replaced = replace_common_expr( expr, id_array, expr_stats, common_exprs, alias_generator, - ) + )?; + // remember if this expression was actually replaced + transformed |= replaced.transformed; + Ok(replaced.data) }) .collect::>>() }) .collect::>>() + .map(|rewritten_exprs_list| { + // propagate back transformed information + Transformed::new_transformed(rewritten_exprs_list, transformed) + }) } /// Rewrites the expression in `exprs_list` with common sub-expressions @@ -161,13 +174,15 @@ impl CommonSubexprEliminate { /// common sub-expressions that were used fn rewrite_expr( &self, - exprs_list: &[&[Expr]], + exprs_list: Vec>, arrays_list: &[&[IdArray]], - input: &LogicalPlan, + input: LogicalPlan, expr_stats: &ExprStats, config: &dyn OptimizerConfig, - ) -> Result<(Vec>, LogicalPlan)> { + ) -> Result>, LogicalPlan)>> { + let mut transformed = false; let mut common_exprs = CommonExprs::new(); + let rewrite_exprs = self.rewrite_exprs_list( exprs_list, arrays_list, @@ -175,115 +190,193 @@ impl CommonSubexprEliminate { &mut common_exprs, &config.alias_generator(), )?; + transformed |= rewrite_exprs.transformed; - let mut new_input = self - .try_optimize(input, config)? - .unwrap_or_else(|| input.clone()); + let new_input = self.rewrite(input, config)?; + transformed |= new_input.transformed; + let mut new_input = new_input.data; if !common_exprs.is_empty() { + assert!(transformed); new_input = build_common_expr_project_plan(new_input, common_exprs)?; } - Ok((rewrite_exprs, new_input)) + // return the transformed information + + Ok(Transformed::new_transformed( + (rewrite_exprs.data, new_input), + transformed, + )) } - fn try_optimize_window( + fn try_optimize_proj( &self, - window: &Window, + projection: Projection, config: &dyn OptimizerConfig, - ) -> Result { - let mut window_exprs = vec![]; - let mut arrays_per_window = vec![]; - let mut expr_stats = ExprStats::new(); - - // Get all window expressions inside the consecutive window operators. - // Consecutive window expressions may refer to same complex expression. - // If same complex expression is referred more than once by subsequent `WindowAggr`s, - // we can cache complex expression by evaluating it with a projection before the - // first WindowAggr. - // This enables us to cache complex expression "c3+c4" for following plan: - // WindowAggr: windowExpr=[[sum(c9) ORDER BY [c3 + c4] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] - // --WindowAggr: windowExpr=[[sum(c9) ORDER BY [c3 + c4] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] - // where, it is referred once by each `WindowAggr` (total of 2) in the plan. - let mut plan = LogicalPlan::Window(window.clone()); - while let LogicalPlan::Window(window) = plan { - let Window { - input, window_expr, .. - } = window; - plan = input.as_ref().clone(); + ) -> Result> { + let Projection { + expr, + input, + schema, + .. + } = projection; + let input = unwrap_arc(input); + self.try_unary_plan(expr, input, config)? + .map_data(|(new_expr, new_input)| { + Projection::try_new_with_schema(new_expr, Arc::new(new_input), schema) + .map(LogicalPlan::Projection) + }) + } + fn try_optimize_sort( + &self, + sort: Sort, + config: &dyn OptimizerConfig, + ) -> Result> { + let Sort { expr, input, fetch } = sort; + let input = unwrap_arc(input); + let new_sort = self.try_unary_plan(expr, input, config)?.update_data( + |(new_expr, new_input)| { + LogicalPlan::Sort(Sort { + expr: new_expr, + input: Arc::new(new_input), + fetch, + }) + }, + ); + Ok(new_sort) + } - let arrays = to_arrays(&window_expr, &mut expr_stats, ExprMask::Normal)?; + fn try_optimize_filter( + &self, + filter: Filter, + config: &dyn OptimizerConfig, + ) -> Result> { + let Filter { + predicate, input, .. + } = filter; + let input = unwrap_arc(input); + let expr = vec![predicate]; + self.try_unary_plan(expr, input, config)? + .transform_data(|(mut new_expr, new_input)| { + assert_eq!(new_expr.len(), 1); // passed in vec![predicate] + let new_predicate = new_expr.pop().unwrap(); + Ok(Filter::remove_aliases(new_predicate)? + .update_data(|new_predicate| (new_predicate, new_input))) + })? + .map_data(|(new_predicate, new_input)| { + Filter::try_new(new_predicate, Arc::new(new_input)) + .map(LogicalPlan::Filter) + }) + } - window_exprs.push(window_expr); - arrays_per_window.push(arrays); - } + fn try_optimize_window( + &self, + window: Window, + config: &dyn OptimizerConfig, + ) -> Result> { + // collect all window expressions from any number of LogicalPlanWindow + let ConsecutiveWindowExprs { + window_exprs, + arrays_per_window, + expr_stats, + plan, + } = ConsecutiveWindowExprs::try_new(window)?; - let mut window_exprs = window_exprs - .iter() - .map(|expr| expr.as_slice()) - .collect::>(); let arrays_per_window = arrays_per_window .iter() .map(|arrays| arrays.as_slice()) .collect::>(); + // save the original names + let name_preserver = NamePreserver::new(&plan); + let mut saved_names = window_exprs + .iter() + .map(|exprs| { + exprs + .iter() + .map(|expr| name_preserver.save(expr)) + .collect::>>() + }) + .collect::>>()?; + assert_eq!(window_exprs.len(), arrays_per_window.len()); - let (mut new_expr, new_input) = self.rewrite_expr( - &window_exprs, + let num_window_exprs = window_exprs.len(); + let rewritten_window_exprs = self.rewrite_expr( + window_exprs, &arrays_per_window, - &plan, + plan, &expr_stats, config, )?; - assert_eq!(window_exprs.len(), new_expr.len()); + let transformed = rewritten_window_exprs.transformed; + + let (mut new_expr, new_input) = rewritten_window_exprs.data; - // Construct consecutive window operator, with their corresponding new window expressions. - plan = new_input; - while let Some(new_window_expr) = new_expr.pop() { - // Since `new_expr` and `window_exprs` length are same. We can safely `.unwrap` here. - let orig_window_expr = window_exprs.pop().unwrap(); - assert_eq!(new_window_expr.len(), orig_window_expr.len()); + let mut plan = new_input; - // Rename new re-written window expressions with original name (by giving alias) - // Otherwise we may receive schema error, in subsequent operators. + // Construct consecutive window operator, with their corresponding new + // window expressions. + // + // Note this iterates over, `new_expr` and `saved_names` which are the + // same length, in reverse order + assert_eq!(num_window_exprs, new_expr.len()); + assert_eq!(num_window_exprs, saved_names.len()); + while let (Some(new_window_expr), Some(saved_names)) = + (new_expr.pop(), saved_names.pop()) + { + assert_eq!(new_window_expr.len(), saved_names.len()); + + // Rename re-written window expressions with original name, to + // preserve the output schema let new_window_expr = new_window_expr .into_iter() - .zip(orig_window_expr.iter()) - .map(|(new_window_expr, window_expr)| { - let original_name = window_expr.name_for_alias()?; - new_window_expr.alias_if_changed(original_name) - }) + .zip(saved_names.into_iter()) + .map(|(new_window_expr, saved_name)| saved_name.restore(new_window_expr)) .collect::>>()?; plan = LogicalPlan::Window(Window::try_new(new_window_expr, Arc::new(plan))?); } - Ok(plan) + Ok(Transformed::new_transformed(plan, transformed)) } fn try_optimize_aggregate( &self, - aggregate: &Aggregate, + aggregate: Aggregate, config: &dyn OptimizerConfig, - ) -> Result { + ) -> Result> { let Aggregate { group_expr, aggr_expr, input, + schema: orig_schema, .. } = aggregate; let mut expr_stats = ExprStats::new(); + // track transformed information + let mut transformed = false; + // rewrite inputs - let group_arrays = to_arrays(group_expr, &mut expr_stats, ExprMask::Normal)?; - let aggr_arrays = to_arrays(aggr_expr, &mut expr_stats, ExprMask::Normal)?; + let group_arrays = to_arrays(&group_expr, &mut expr_stats, ExprMask::Normal)?; + let aggr_arrays = to_arrays(&aggr_expr, &mut expr_stats, ExprMask::Normal)?; + + let name_perserver = NamePreserver::new_for_projection(); + let saved_names = aggr_expr + .iter() + .map(|expr| name_perserver.save(expr)) + .collect::>>()?; - let (mut new_expr, new_input) = self.rewrite_expr( - &[group_expr, aggr_expr], + // rewrite both group exprs and aggr_expr + let rewritten = self.rewrite_expr( + vec![group_expr, aggr_expr], &[&group_arrays, &aggr_arrays], - input, + unwrap_arc(input), &expr_stats, config, )?; + transformed |= rewritten.transformed; + let (mut new_expr, new_input) = rewritten.data; + // note the reversed pop order. let new_aggr_expr = pop_expr(&mut new_expr)?; let new_group_expr = pop_expr(&mut new_expr)?; @@ -296,108 +389,208 @@ impl CommonSubexprEliminate { &mut expr_stats, ExprMask::NormalAndAggregates, )?; - let mut common_exprs = CommonExprs::new(); - let mut rewritten = self.rewrite_exprs_list( - &[&new_aggr_expr], + let mut common_exprs = IndexMap::new(); + let mut rewritten_exprs = self.rewrite_exprs_list( + vec![new_aggr_expr.clone()], &[&aggr_arrays], &expr_stats, &mut common_exprs, &config.alias_generator(), )?; - let rewritten = pop_expr(&mut rewritten)?; + transformed |= rewritten_exprs.transformed; + let rewritten = pop_expr(&mut rewritten_exprs.data)?; if common_exprs.is_empty() { // Alias aggregation expressions if they have changed let new_aggr_expr = new_aggr_expr - .iter() - .zip(aggr_expr.iter()) - .map(|(new_expr, old_expr)| { - new_expr.clone().alias_if_changed(old_expr.display_name()?) - }) + .into_iter() + .zip(saved_names.into_iter()) + .map(|(new_expr, saved_name)| saved_name.restore(new_expr)) .collect::>>()?; - // Since group_epxr changes, schema changes also. Use try_new method. - Aggregate::try_new(Arc::new(new_input), new_group_expr, new_aggr_expr) - .map(LogicalPlan::Aggregate) - } else { - let mut agg_exprs = common_exprs - .into_values() - .map(|(expr, expr_alias)| expr.alias(expr_alias)) - .collect::>(); - - let mut proj_exprs = vec![]; - for expr in &new_group_expr { - extract_expressions(expr, &new_input_schema, &mut proj_exprs)? - } - for (expr_rewritten, expr_orig) in rewritten.into_iter().zip(new_aggr_expr) { - if expr_rewritten == expr_orig { - if let Expr::Alias(Alias { expr, name, .. }) = expr_rewritten { - agg_exprs.push(expr.alias(&name)); - proj_exprs.push(Expr::Column(Column::from_name(name))); - } else { - let expr_alias = config.alias_generator().next(CSE_PREFIX); - let (qualifier, field) = - expr_rewritten.to_field(&new_input_schema)?; - let out_name = qualified_name(qualifier.as_ref(), field.name()); - - agg_exprs.push(expr_rewritten.alias(&expr_alias)); - proj_exprs.push( - Expr::Column(Column::from_name(expr_alias)).alias(out_name), - ); - } + // Since group_expr may have changed, schema may also. Use try_new method. + let new_agg = if transformed { + Aggregate::try_new(Arc::new(new_input), new_group_expr, new_aggr_expr)? + } else { + Aggregate::try_new_with_schema( + Arc::new(new_input), + new_group_expr, + new_aggr_expr, + orig_schema, + )? + }; + let new_agg = LogicalPlan::Aggregate(new_agg); + return Ok(Transformed::new_transformed(new_agg, transformed)); + } + let mut agg_exprs = common_exprs + .into_values() + .map(|(expr, expr_alias)| expr.alias(expr_alias)) + .collect::>(); + + let mut proj_exprs = vec![]; + for expr in &new_group_expr { + extract_expressions(expr, &new_input_schema, &mut proj_exprs)? + } + for (expr_rewritten, expr_orig) in rewritten.into_iter().zip(new_aggr_expr) { + if expr_rewritten == expr_orig { + if let Expr::Alias(Alias { expr, name, .. }) = expr_rewritten { + agg_exprs.push(expr.alias(&name)); + proj_exprs.push(Expr::Column(Column::from_name(name))); } else { - proj_exprs.push(expr_rewritten); + let expr_alias = config.alias_generator().next(CSE_PREFIX); + let (qualifier, field) = + expr_rewritten.to_field(&new_input_schema)?; + let out_name = qualified_name(qualifier.as_ref(), field.name()); + + agg_exprs.push(expr_rewritten.alias(&expr_alias)); + proj_exprs.push( + Expr::Column(Column::from_name(expr_alias)).alias(out_name), + ); } + } else { + proj_exprs.push(expr_rewritten); } + } - let agg = LogicalPlan::Aggregate(Aggregate::try_new( - Arc::new(new_input), - new_group_expr, - agg_exprs, - )?); + let agg = LogicalPlan::Aggregate(Aggregate::try_new( + Arc::new(new_input), + new_group_expr, + agg_exprs, + )?); - Ok(LogicalPlan::Projection(Projection::try_new( - proj_exprs, - Arc::new(agg), - )?)) - } + Projection::try_new(proj_exprs, Arc::new(agg)) + .map(LogicalPlan::Projection) + .map(Transformed::yes) } + /// Rewrites the expr list and input to remove common subexpressions + /// + /// # Parameters + /// + /// * `exprs`: List of expressions in the node + /// * `input`: input plan (that produces the columns referred to in `exprs`) + /// + /// # Return value + /// + /// Returns `(rewritten_exprs, new_input)`. `new_input` is either: + /// + /// 1. The original `input` of no common subexpressions were extracted + /// 2. A newly added projection on top of the original input + /// that computes the common subexpressions fn try_unary_plan( &self, - plan: &LogicalPlan, + expr: Vec, + input: LogicalPlan, config: &dyn OptimizerConfig, - ) -> Result { - let expr = plan.expressions(); - let inputs = plan.inputs(); - let input = inputs[0]; + ) -> Result, LogicalPlan)>> { let mut expr_stats = ExprStats::new(); - - // Visit expr list and build expr identifier to occuring count map (`expr_stats`). let arrays = to_arrays(&expr, &mut expr_stats, ExprMask::Normal)?; - let (mut new_expr, new_input) = - self.rewrite_expr(&[&expr], &[&arrays], input, &expr_stats, config)?; + self.rewrite_expr(vec![expr], &[&arrays], input, &expr_stats, config)? + .map_data(|(mut new_expr, new_input)| { + assert_eq!(new_expr.len(), 1); + Ok((new_expr.pop().unwrap(), new_input)) + }) + } +} - plan.with_new_exprs(pop_expr(&mut new_expr)?, vec![new_input]) +/// Get all window expressions inside the consecutive window operators. +/// +/// Returns the window expressions, and the input to the deepest child +/// LogicalPlan. +/// +/// For example, if the input widnow looks like +/// +/// ```text +/// LogicalPlan::Window(exprs=[a, b, c]) +/// LogicalPlan::Window(exprs=[d]) +/// InputPlan +/// ``` +/// +/// Returns: +/// * `window_exprs`: `[a, b, c, d]` +/// * InputPlan +/// +/// Consecutive window expressions may refer to same complex expression. +/// +/// If same complex expression is referred more than once by subsequent +/// `WindowAggr`s, we can cache complex expression by evaluating it with a +/// projection before the first WindowAggr. +/// +/// This enables us to cache complex expression "c3+c4" for following plan: +/// +/// ```text +/// WindowAggr: windowExpr=[[sum(c9) ORDER BY [c3 + c4] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +/// --WindowAggr: windowExpr=[[sum(c9) ORDER BY [c3 + c4] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +/// ``` +/// +/// where, it is referred once by each `WindowAggr` (total of 2) in the plan. +struct ConsecutiveWindowExprs { + window_exprs: Vec>, + /// result of calling `to_arrays` on each set of window exprs + arrays_per_window: Vec>>, + expr_stats: ExprStats, + /// input plan to the window + plan: LogicalPlan, +} + +impl ConsecutiveWindowExprs { + fn try_new(window: Window) -> Result { + let mut window_exprs = vec![]; + let mut arrays_per_window = vec![]; + let mut expr_stats = ExprStats::new(); + + let mut plan = LogicalPlan::Window(window); + while let LogicalPlan::Window(Window { + input, window_expr, .. + }) = plan + { + plan = unwrap_arc(input); + + let arrays = to_arrays(&window_expr, &mut expr_stats, ExprMask::Normal)?; + + window_exprs.push(window_expr); + arrays_per_window.push(arrays); + } + + Ok(Self { + window_exprs, + arrays_per_window, + expr_stats, + plan, + }) } } impl OptimizerRule for CommonSubexprEliminate { fn try_optimize( &self, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, + _plan: &LogicalPlan, + _config: &dyn OptimizerConfig, ) -> Result> { + internal_err!("Should have called CommonSubexprEliminate::rewrite") + } + + fn supports_rewrite(&self) -> bool { + true + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::TopDown) + } + + fn rewrite( + &self, + plan: LogicalPlan, + config: &dyn OptimizerConfig, + ) -> Result> { + let original_schema = Arc::clone(plan.schema()); + let optimized_plan = match plan { - LogicalPlan::Projection(_) - | LogicalPlan::Sort(_) - | LogicalPlan::Filter(_) => Some(self.try_unary_plan(plan, config)?), - LogicalPlan::Window(window) => { - Some(self.try_optimize_window(window, config)?) - } - LogicalPlan::Aggregate(aggregate) => { - Some(self.try_optimize_aggregate(aggregate, config)?) - } + LogicalPlan::Projection(proj) => self.try_optimize_proj(proj, config)?, + LogicalPlan::Sort(sort) => self.try_optimize_sort(sort, config)?, + LogicalPlan::Filter(filter) => self.try_optimize_filter(filter, config)?, + LogicalPlan::Window(window) => self.try_optimize_window(window, config)?, + LogicalPlan::Aggregate(agg) => self.try_optimize_aggregate(agg, config)?, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_) | LogicalPlan::Repartition(_) @@ -420,21 +613,19 @@ impl OptimizerRule for CommonSubexprEliminate { | LogicalPlan::Unnest(_) | LogicalPlan::RecursiveQuery(_) | LogicalPlan::Prepare(_) => { - // apply the optimization to all inputs of the plan - utils::optimize_children(self, plan, config)? + // ApplyOrder::TopDown handles recursion + Transformed::no(plan) } }; - let original_schema = plan.schema(); - match optimized_plan { - Some(optimized_plan) if optimized_plan.schema() != original_schema => { - // add an additional projection if the output schema changed. - Ok(Some(build_recover_project_plan( - original_schema, - optimized_plan, - )?)) - } - plan => Ok(plan), + // If we rewrote the plan, ensure the schema stays the same + if optimized_plan.transformed && optimized_plan.data.schema() != &original_schema + { + optimized_plan.map_data(|optimized_plan| { + build_recover_project_plan(&original_schema, optimized_plan) + }) + } else { + Ok(optimized_plan) } } @@ -459,22 +650,29 @@ impl CommonSubexprEliminate { fn pop_expr(new_expr: &mut Vec>) -> Result> { new_expr .pop() - .ok_or_else(|| DataFusionError::Internal("Failed to pop expression".to_string())) + .ok_or_else(|| internal_datafusion_err!("Failed to pop expression")) } +/// Returns the identifier list for each element in `exprs` +/// +/// Returns and array with 1 element for each input expr in `exprs` +/// +/// Each element is itself the result of [`expr_to_identifier`] for that expr +/// (e.g. the identifiers for each node in the tree) fn to_arrays( - expr: &[Expr], + exprs: &[Expr], expr_stats: &mut ExprStats, expr_mask: ExprMask, ) -> Result> { - expr.iter() + exprs + .iter() .map(|e| { let mut id_array = vec![]; expr_to_identifier(e, expr_stats, &mut id_array, expr_mask)?; Ok(id_array) }) - .collect::>>() + .collect() } /// Build the "intermediate" projection plan that evaluates the extracted common @@ -506,10 +704,7 @@ fn build_common_expr_project_plan( } } - Ok(LogicalPlan::Projection(Projection::try_new( - project_exprs, - Arc::new(input), - )?)) + Projection::try_new(project_exprs, Arc::new(input)).map(LogicalPlan::Projection) } /// Build the projection plan to eliminate unnecessary columns produced by @@ -522,10 +717,7 @@ fn build_recover_project_plan( input: LogicalPlan, ) -> Result { let col_exprs = schema.iter().map(Expr::from).collect(); - Ok(LogicalPlan::Projection(Projection::try_new( - col_exprs, - Arc::new(input), - )?)) + Projection::try_new(col_exprs, Arc::new(input)).map(LogicalPlan::Projection) } fn extract_expressions( @@ -807,7 +999,7 @@ fn replace_common_expr( expr_stats: &ExprStats, common_exprs: &mut CommonExprs, alias_generator: &AliasGenerator, -) -> Result { +) -> Result> { expr.rewrite(&mut CommonSubexprRewriter { expr_stats, id_array, @@ -816,7 +1008,6 @@ fn replace_common_expr( alias_counter: 0, alias_generator, }) - .data() } #[cfg(test)] @@ -839,18 +1030,36 @@ mod test { use super::*; + fn assert_non_optimized_plan_eq( + expected: &str, + plan: LogicalPlan, + config: Option<&dyn OptimizerConfig>, + ) { + assert_eq!(expected, format!("{plan:?}"), "Unexpected starting plan"); + let optimizer = CommonSubexprEliminate {}; + let default_config = OptimizerContext::new(); + let config = config.unwrap_or(&default_config); + let optimized_plan = optimizer.rewrite(plan, config).unwrap(); + assert!(!optimized_plan.transformed, "unexpectedly optimize plan"); + let optimized_plan = optimized_plan.data; + assert_eq!( + expected, + format!("{optimized_plan:?}"), + "Unexpected optimized plan" + ); + } + fn assert_optimized_plan_eq( expected: &str, - plan: &LogicalPlan, + plan: LogicalPlan, config: Option<&dyn OptimizerConfig>, ) { let optimizer = CommonSubexprEliminate {}; let default_config = OptimizerContext::new(); let config = config.unwrap_or(&default_config); - let optimized_plan = optimizer - .try_optimize(plan, config) - .unwrap() - .expect("failed to optimize plan"); + let optimized_plan = optimizer.rewrite(plan, config).unwrap(); + assert!(optimized_plan.transformed, "failed to optimize plan"); + let optimized_plan = optimized_plan.data; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(expected, formatted_plan); } @@ -933,7 +1142,7 @@ mod test { \n Projection: test.a * (Int32(1) - test.b) AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -953,7 +1162,7 @@ mod test { \n Projection: test.a + test.b AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1006,7 +1215,7 @@ mod test { \n Aggregate: groupBy=[[]], aggr=[[AVG(test.a) AS __common_expr_1, my_agg(test.a) AS __common_expr_2, AVG(test.b) AS col3, AVG(test.c) AS __common_expr_3, my_agg(test.b) AS col6, my_agg(test.c) AS __common_expr_4]]\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); // test: trafo after aggregate let plan = LogicalPlanBuilder::from(table_scan.clone()) @@ -1025,7 +1234,7 @@ mod test { \n Aggregate: groupBy=[[]], aggr=[[AVG(test.a) AS __common_expr_1, my_agg(test.a) AS __common_expr_2]]\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); // test: transformation before aggregate let plan = LogicalPlanBuilder::from(table_scan.clone()) @@ -1042,7 +1251,7 @@ mod test { \n Projection: UInt32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); // test: common between agg and group let plan = LogicalPlanBuilder::from(table_scan.clone()) @@ -1059,7 +1268,7 @@ mod test { \n Projection: UInt32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); // test: all mixed let plan = LogicalPlanBuilder::from(table_scan) @@ -1081,7 +1290,7 @@ mod test { \n Projection: UInt32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1108,7 +1317,7 @@ mod test { \n Projection: UInt32(1) + table.test.col.a AS __common_expr_1, table.test.col.a\ \n TableScan: table.test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1128,7 +1337,7 @@ mod test { \n Projection: Int32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1144,7 +1353,7 @@ mod test { let expected = "Projection: Int32(1) + test.a, test.a + Int32(1)\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_non_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1162,7 +1371,7 @@ mod test { \n Projection: Int32(1) + test.a, test.a\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_non_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1257,10 +1466,9 @@ mod test { .build() .unwrap(); let rule = CommonSubexprEliminate {}; - let optimized_plan = rule - .try_optimize(&plan, &OptimizerContext::new()) - .unwrap() - .unwrap(); + let optimized_plan = rule.rewrite(plan, &OptimizerContext::new()).unwrap(); + assert!(!optimized_plan.transformed); + let optimized_plan = optimized_plan.data; let schema = optimized_plan.schema(); let fields_with_datatypes: Vec<_> = schema @@ -1299,7 +1507,7 @@ mod test { \n Projection: Int32(1) + test.a AS __common_expr_1, test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, None); + assert_optimized_plan_eq(expected, plan, None); Ok(()) } @@ -1365,7 +1573,7 @@ mod test { \n Projection: test.a + test.b AS __common_expr_1, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, Some(config)); + assert_optimized_plan_eq(expected, plan, Some(config)); let config = &OptimizerContext::new(); let _common_expr_1 = config.alias_generator().next(CSE_PREFIX); @@ -1388,7 +1596,7 @@ mod test { \n Projection: test.a + test.b AS __common_expr_2, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(expected, &plan, Some(config)); + assert_optimized_plan_eq(expected, plan, Some(config)); Ok(()) } From 0f80b9261f389241f1d76449940faccf7e237e80 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 20 Jun 2024 00:15:25 +0800 Subject: [PATCH 28/54] feat: Update documentation link in physical optimizer module (#11002) --- datafusion/core/src/physical_optimizer/optimizer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index e3b60a0cca80..6880a5433943 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -40,7 +40,7 @@ use crate::{error::Result, physical_plan::ExecutionPlan}; /// Use [`SessionState::add_physical_optimizer_rule`] to register additional /// `PhysicalOptimizerRule`s. /// -/// [`SessionState::add_physical_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionState.html#method.add_physical_optimizer_rule +/// [`SessionState::add_physical_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_physical_optimizer_rule pub trait PhysicalOptimizerRule { /// Rewrite `plan` to an optimized form fn optimize( From 61e2ddbf29f7bf5d48f748554a00bb050a0a6cc2 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 20 Jun 2024 04:43:36 +0800 Subject: [PATCH 29/54] fix push down logic for unnest (#10991) Signed-off-by: jayzhan211 --- datafusion/optimizer/src/push_down_filter.rs | 13 +++++- .../test_files/push_down_filter.slt | 43 +++++++++++++++++-- 2 files changed, 51 insertions(+), 5 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 8dc71b70e22b..295039af2f19 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -32,7 +32,9 @@ use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{ CrossJoin, Join, JoinType, LogicalPlan, TableScan, Union, }; -use datafusion_expr::utils::{conjunction, split_conjunction, split_conjunction_owned}; +use datafusion_expr::utils::{ + conjunction, expr_to_columns, split_conjunction, split_conjunction_owned, +}; use datafusion_expr::{ and, build_join_schema, or, BinaryExpr, Expr, Filter, LogicalPlanBuilder, Operator, Projection, TableProviderFilterPushDown, @@ -707,6 +709,15 @@ impl OptimizerRule for PushDownFilter { } } LogicalPlan::Unnest(mut unnest) => { + // collect all the Expr::Column in predicate recursively + let mut accum: HashSet = HashSet::new(); + expr_to_columns(&filter.predicate, &mut accum)?; + + if unnest.exec_columns.iter().any(|c| accum.contains(c)) { + filter.input = Arc::new(LogicalPlan::Unnest(unnest)); + return Ok(Transformed::no(LogicalPlan::Filter(filter))); + } + // Unnest is built above Projection, so we only take Projection into consideration match unwrap_arc(unnest.input) { LogicalPlan::Projection(projection) => { diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 1d72e11f4352..368243f9669b 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -42,10 +42,45 @@ logical_plan 04)------Filter: v.column1 = Int64(2) 05)--------TableScan: v projection=[column1, column2] -# TODO: fix the query -query error DataFusion error: External error: Arrow error: Invalid argument error: Invalid comparison operation: List\(Field \{ name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: \{\} \}\) > Int64 +query I select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; +---- +4 +5 # test push down filter for unnest with filter on unnest column -# query TT -# explain select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; +query TT +explain select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; +---- +logical_plan +01)Projection: unnest(v.column2) AS uc2 +02)--Filter: unnest(v.column2) > Int64(3) +03)----Projection: unnest(v.column2) +04)------Unnest: lists[unnest(v.column2)] structs[] +05)--------Projection: v.column2 AS unnest(v.column2), v.column1 +06)----------TableScan: v projection=[column1, column2] + +statement ok +drop table v; + +# test with unnest struct, should not push down filter +statement ok +CREATE TABLE d AS VALUES(1,[named_struct('a', 1, 'b', 2)]),(2,[named_struct('a', 3, 'b', 4), named_struct('a', 5, 'b', 6)]); + +query I? +select * from (select column1, unnest(column2) as o from d) where o['a'] = 1; +---- +1 {a: 1, b: 2} + +query TT +explain select * from (select column1, unnest(column2) as o from d) where o['a'] = 1; +---- +logical_plan +01)Projection: d.column1, unnest(d.column2) AS o +02)--Filter: get_field(unnest(d.column2), Utf8("a")) = Int64(1) +03)----Unnest: lists[unnest(d.column2)] structs[] +04)------Projection: d.column1, d.column2 AS unnest(d.column2) +05)--------TableScan: d projection=[column1, column2] + +statement ok +drop table d; From 1e7c38b4f0dfbd77c188a6085400af76d22dbe30 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 19 Jun 2024 20:38:35 -0400 Subject: [PATCH 30/54] Minor: add test for pushdown past unnest (#11017) --- .../test_files/push_down_filter.slt | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 368243f9669b..5029ab170a18 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -60,6 +60,25 @@ logical_plan 05)--------Projection: v.column2 AS unnest(v.column2), v.column1 06)----------TableScan: v projection=[column1, column2] +query II +select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; +---- +4 2 +5 2 + +# Could push the filter (column1 = 2) down below unnest +# https://github.com/apache/datafusion/issues/11016 +query TT +explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; +---- +logical_plan +01)Projection: unnest(v.column2) AS uc2, v.column1 +02)--Filter: unnest(v.column2) > Int64(3) AND v.column1 = Int64(2) +03)----Unnest: lists[unnest(v.column2)] structs[] +04)------Projection: v.column2 AS unnest(v.column2), v.column1 +05)--------TableScan: v projection=[column1, column2] + + statement ok drop table v; @@ -82,5 +101,7 @@ logical_plan 04)------Projection: d.column1, d.column2 AS unnest(d.column2) 05)--------TableScan: d projection=[column1, column2] + + statement ok drop table d; From 5bfc11ba4ac4f11eaf9793c668e4a064fb697e6e Mon Sep 17 00:00:00 2001 From: jcsherin Date: Thu, 20 Jun 2024 06:11:41 +0530 Subject: [PATCH 31/54] Update docs for `protoc` minimum installed version (#11006) * bumps `protoc` minimum version from 3.12 -> 3.15 * Minor: Fix link for Protoc installation * minor: removes reference to 3.12 --- datafusion/proto/CONTRIBUTING.md | 2 +- docs/source/contributor-guide/getting_started.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/proto/CONTRIBUTING.md b/datafusion/proto/CONTRIBUTING.md index f124c233d04f..db3658c72610 100644 --- a/datafusion/proto/CONTRIBUTING.md +++ b/datafusion/proto/CONTRIBUTING.md @@ -29,4 +29,4 @@ valid installation of [protoc] (see [installation instructions] for details). ``` [protoc]: https://github.com/protocolbuffers/protobuf#protocol-compiler-installation -[installation instructions]: https://datafusion.apache.org/contributor-guide/#protoc-installation +[installation instructions]: https://datafusion.apache.org/contributor-guide/getting_started.html#protoc-installation diff --git a/docs/source/contributor-guide/getting_started.md b/docs/source/contributor-guide/getting_started.md index 64d5a0d43d5d..696d6d3a0fe2 100644 --- a/docs/source/contributor-guide/getting_started.md +++ b/docs/source/contributor-guide/getting_started.md @@ -50,11 +50,11 @@ $ pacman -S protobuf $ brew install protobuf ``` -You will want to verify the version installed is `3.12` or greater, which introduced support for explicit [field presence](https://github.com/protocolbuffers/protobuf/blob/v3.12.0/docs/field_presence.md). Older versions may fail to compile. +You will want to verify the version installed is `3.15` or greater, which has support for explicit [field presence](https://github.com/protocolbuffers/protobuf/blob/v3.15.0/docs/field_presence.md). Older versions may fail to compile. ```shell $ protoc --version -libprotoc 3.12.4 +libprotoc 3.15.0 ``` Alternatively a binary release can be downloaded from the [Release Page](https://github.com/protocolbuffers/protobuf/releases) or [built from source](https://github.com/protocolbuffers/protobuf/blob/main/src/README.md). From 89def2c6e5f32f15dd80dd19d11b3087e1647310 Mon Sep 17 00:00:00 2001 From: jcsherin Date: Thu, 20 Jun 2024 16:29:59 +0530 Subject: [PATCH 32/54] Convert `bool_and` & `bool_or` to UDAF (#11009) * Port `bool_and` and `bool_or` to `AggregateUDFImpl` * Remove trait methods with default implementation * Add `bool_or_udaf` * Register `bool_and` and `bool_or` * Remove from `physical-expr` * Add expressions to logical plan roundtrip test * minor: remove methods with default implementation * Removes redundant tests * Removes hard-coded function names --- datafusion/expr/src/aggregate_function.rs | 15 - .../expr/src/type_coercion/aggregates.rs | 16 - .../functions-aggregate/src/bool_and_or.rs | 343 +++++++++++++++ datafusion/functions-aggregate/src/lib.rs | 6 +- .../src/aggregate/bool_and_or.rs | 394 ------------------ .../physical-expr/src/aggregate/build_in.rs | 56 +-- .../src/aggregate/groups_accumulator/mod.rs | 3 - datafusion/physical-expr/src/aggregate/mod.rs | 1 - .../physical-expr/src/expressions/mod.rs | 1 - datafusion/proto/proto/datafusion.proto | 4 +- datafusion/proto/src/generated/pbjson.rs | 6 - datafusion/proto/src/generated/prost.rs | 8 +- .../proto/src/logical_plan/from_proto.rs | 2 - datafusion/proto/src/logical_plan/to_proto.rs | 4 - .../proto/src/physical_plan/to_proto.rs | 12 +- .../tests/cases/roundtrip_logical_plan.rs | 6 +- 16 files changed, 362 insertions(+), 515 deletions(-) create mode 100644 datafusion/functions-aggregate/src/bool_and_or.rs delete mode 100644 datafusion/physical-expr/src/aggregate/bool_and_or.rs diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 1cde1c5050a8..967ccc0b0866 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -47,10 +47,6 @@ pub enum AggregateFunction { Correlation, /// Grouping Grouping, - /// Bool And - BoolAnd, - /// Bool Or - BoolOr, } impl AggregateFunction { @@ -64,8 +60,6 @@ impl AggregateFunction { NthValue => "NTH_VALUE", Correlation => "CORR", Grouping => "GROUPING", - BoolAnd => "BOOL_AND", - BoolOr => "BOOL_OR", } } } @@ -82,8 +76,6 @@ impl FromStr for AggregateFunction { Ok(match name { // general "avg" => AggregateFunction::Avg, - "bool_and" => AggregateFunction::BoolAnd, - "bool_or" => AggregateFunction::BoolOr, "max" => AggregateFunction::Max, "mean" => AggregateFunction::Avg, "min" => AggregateFunction::Min, @@ -128,9 +120,6 @@ impl AggregateFunction { // The coerced_data_types is same with input_types. Ok(coerced_data_types[0].clone()) } - AggregateFunction::BoolAnd | AggregateFunction::BoolOr => { - Ok(DataType::Boolean) - } AggregateFunction::Correlation => { correlation_return_type(&coerced_data_types[0]) } @@ -179,10 +168,6 @@ impl AggregateFunction { .collect::>(); Signature::uniform(1, valid, Volatility::Immutable) } - AggregateFunction::BoolAnd | AggregateFunction::BoolOr => { - Signature::uniform(1, vec![DataType::Boolean], Volatility::Immutable) - } - AggregateFunction::Avg => { Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable) } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index abe6d8b1823d..428fc99070d2 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -121,18 +121,6 @@ pub fn coerce_types( }; Ok(vec![v]) } - AggregateFunction::BoolAnd | AggregateFunction::BoolOr => { - // Refer to https://www.postgresql.org/docs/8.2/functions-aggregate.html doc - // smallint, int, bigint, real, double precision, decimal, or interval. - if !is_bool_and_or_support_arg_type(&input_types[0]) { - return plan_err!( - "The function {:?} does not support inputs of type {:?}.", - agg_fun, - input_types[0] - ); - } - Ok(input_types.to_vec()) - } AggregateFunction::Correlation => { if !is_correlation_support_arg_type(&input_types[0]) { return plan_err!( @@ -319,10 +307,6 @@ pub fn avg_sum_type(arg_type: &DataType) -> Result { } } -pub fn is_bool_and_or_support_arg_type(arg_type: &DataType) -> bool { - matches!(arg_type, DataType::Boolean) -} - pub fn is_sum_support_arg_type(arg_type: &DataType) -> bool { match arg_type { DataType::Dictionary(_, dict_value_type) => { diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs new file mode 100644 index 000000000000..d0028672743e --- /dev/null +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -0,0 +1,343 @@ +// 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. + +//! Defines physical expressions that can evaluated at runtime during query execution + +use std::any::Any; + +use arrow::array::ArrayRef; +use arrow::array::BooleanArray; +use arrow::compute::bool_and as compute_bool_and; +use arrow::compute::bool_or as compute_bool_or; +use arrow::datatypes::DataType; +use arrow::datatypes::Field; + +use datafusion_common::internal_err; +use datafusion_common::{downcast_value, not_impl_err}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, GroupsAccumulator, ReversedUDAF, Signature, Volatility, +}; + +use datafusion_physical_expr_common::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; + +// returns the new value after bool_and/bool_or with the new values, taking nullability into account +macro_rules! typed_bool_and_or_batch { + ($VALUES:expr, $ARRAYTYPE:ident, $SCALAR:ident, $OP:ident) => {{ + let array = downcast_value!($VALUES, $ARRAYTYPE); + let delta = $OP(array); + Ok(ScalarValue::$SCALAR(delta)) + }}; +} + +// bool_and/bool_or the array and returns a ScalarValue of its corresponding type. +macro_rules! bool_and_or_batch { + ($VALUES:expr, $OP:ident) => {{ + match $VALUES.data_type() { + DataType::Boolean => { + typed_bool_and_or_batch!($VALUES, BooleanArray, Boolean, $OP) + } + e => { + return internal_err!( + "Bool and/Bool or is not expected to receive the type {e:?}" + ); + } + } + }}; +} + +/// dynamically-typed bool_and(array) -> ScalarValue +fn bool_and_batch(values: &ArrayRef) -> Result { + bool_and_or_batch!(values, compute_bool_and) +} + +/// dynamically-typed bool_or(array) -> ScalarValue +fn bool_or_batch(values: &ArrayRef) -> Result { + bool_and_or_batch!(values, compute_bool_or) +} + +make_udaf_expr_and_func!( + BoolAnd, + bool_and, + expression, + "The values to combine with `AND`", + bool_and_udaf +); + +make_udaf_expr_and_func!( + BoolOr, + bool_or, + expression, + "The values to combine with `OR`", + bool_or_udaf +); + +/// BOOL_AND aggregate expression +#[derive(Debug)] +pub struct BoolAnd { + signature: Signature, +} + +impl BoolAnd { + fn new() -> Self { + Self { + signature: Signature::uniform( + 1, + vec![DataType::Boolean], + Volatility::Immutable, + ), + } + } +} + +impl Default for BoolAnd { + fn default() -> Self { + Self::new() + } +} + +impl AggregateUDFImpl for BoolAnd { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "bool_and" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _: &[DataType]) -> Result { + Ok(DataType::Boolean) + } + + fn accumulator(&self, _: AccumulatorArgs) -> Result> { + Ok(Box::::default()) + } + + fn state_fields(&self, args: StateFieldsArgs) -> Result> { + Ok(vec![Field::new( + format_state_name(args.name, self.name()), + DataType::Boolean, + true, + )]) + } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + match args.data_type { + DataType::Boolean => { + Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x && y))) + } + _ => not_impl_err!( + "GroupsAccumulator not supported for {} with {}", + args.name, + args.data_type + ), + } + } + + fn aliases(&self) -> &[String] { + &[] + } + + fn create_sliding_accumulator( + &self, + _: AccumulatorArgs, + ) -> Result> { + Ok(Box::::default()) + } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::Insensitive + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } +} + +#[derive(Debug, Default)] +struct BoolAndAccumulator { + acc: Option, +} + +impl Accumulator for BoolAndAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = &values[0]; + self.acc = match (self.acc, bool_and_batch(values)?) { + (None, ScalarValue::Boolean(v)) => v, + (Some(v), ScalarValue::Boolean(None)) => Some(v), + (Some(a), ScalarValue::Boolean(Some(b))) => Some(a && b), + _ => unreachable!(), + }; + Ok(()) + } + + fn evaluate(&mut self) -> Result { + Ok(ScalarValue::Boolean(self.acc)) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![ScalarValue::Boolean(self.acc)]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + self.update_batch(states) + } +} + +/// BOOL_OR aggregate expression +#[derive(Debug, Clone)] +pub struct BoolOr { + signature: Signature, +} + +impl BoolOr { + fn new() -> Self { + Self { + signature: Signature::uniform( + 1, + vec![DataType::Boolean], + Volatility::Immutable, + ), + } + } +} + +impl Default for BoolOr { + fn default() -> Self { + Self::new() + } +} + +impl AggregateUDFImpl for BoolOr { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "bool_or" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _: &[DataType]) -> Result { + Ok(DataType::Boolean) + } + + fn accumulator(&self, _: AccumulatorArgs) -> Result> { + Ok(Box::::default()) + } + + fn state_fields(&self, args: StateFieldsArgs) -> Result> { + Ok(vec![Field::new( + format_state_name(args.name, self.name()), + DataType::Boolean, + true, + )]) + } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + args: AccumulatorArgs, + ) -> Result> { + match args.data_type { + DataType::Boolean => { + Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x || y))) + } + _ => not_impl_err!( + "GroupsAccumulator not supported for {} with {}", + args.name, + args.data_type + ), + } + } + + fn aliases(&self) -> &[String] { + &[] + } + + fn create_sliding_accumulator( + &self, + _: AccumulatorArgs, + ) -> Result> { + Ok(Box::::default()) + } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::Insensitive + } + + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Identical + } +} + +#[derive(Debug, Default)] +struct BoolOrAccumulator { + acc: Option, +} + +impl Accumulator for BoolOrAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = &values[0]; + self.acc = match (self.acc, bool_or_batch(values)?) { + (None, ScalarValue::Boolean(v)) => v, + (Some(v), ScalarValue::Boolean(None)) => Some(v), + (Some(a), ScalarValue::Boolean(Some(b))) => Some(a || b), + _ => unreachable!(), + }; + Ok(()) + } + + fn evaluate(&mut self) -> Result { + Ok(ScalarValue::Boolean(self.acc)) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![ScalarValue::Boolean(self.acc)]) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + self.update_batch(states) + } +} diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 20a8d2c15926..260d6dab31b9 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -70,8 +70,8 @@ pub mod approx_median; pub mod approx_percentile_cont; pub mod approx_percentile_cont_with_weight; pub mod bit_and_or_xor; +pub mod bool_and_or; pub mod string_agg; - use crate::approx_percentile_cont::approx_percentile_cont_udaf; use crate::approx_percentile_cont_with_weight::approx_percentile_cont_with_weight_udaf; use datafusion_common::Result; @@ -89,6 +89,8 @@ pub mod expr_fn { pub use super::bit_and_or_xor::bit_and; pub use super::bit_and_or_xor::bit_or; pub use super::bit_and_or_xor::bit_xor; + pub use super::bool_and_or::bool_and; + pub use super::bool_and_or::bool_or; pub use super::count::count; pub use super::count::count_distinct; pub use super::covariance::covar_pop; @@ -143,6 +145,8 @@ pub fn all_default_aggregate_functions() -> Vec> { bit_and_or_xor::bit_and_udaf(), bit_and_or_xor::bit_or_udaf(), bit_and_or_xor::bit_xor_udaf(), + bool_and_or::bool_and_udaf(), + bool_and_or::bool_or_udaf(), ] } diff --git a/datafusion/physical-expr/src/aggregate/bool_and_or.rs b/datafusion/physical-expr/src/aggregate/bool_and_or.rs deleted file mode 100644 index 341932bd77a4..000000000000 --- a/datafusion/physical-expr/src/aggregate/bool_and_or.rs +++ /dev/null @@ -1,394 +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. - -//! Defines physical expressions that can evaluated at runtime during query execution - -use crate::{AggregateExpr, PhysicalExpr}; -use arrow::datatypes::DataType; -use arrow::{ - array::{ArrayRef, BooleanArray}, - datatypes::Field, -}; -use datafusion_common::{ - downcast_value, internal_err, not_impl_err, DataFusionError, Result, ScalarValue, -}; -use datafusion_expr::{Accumulator, GroupsAccumulator}; -use std::any::Any; -use std::sync::Arc; - -use crate::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; -use crate::aggregate::utils::down_cast_any_ref; -use crate::expressions::format_state_name; -use arrow::array::Array; -use arrow::compute::{bool_and, bool_or}; - -// returns the new value after bool_and/bool_or with the new values, taking nullability into account -macro_rules! typed_bool_and_or_batch { - ($VALUES:expr, $ARRAYTYPE:ident, $SCALAR:ident, $OP:ident) => {{ - let array = downcast_value!($VALUES, $ARRAYTYPE); - let delta = $OP(array); - Ok(ScalarValue::$SCALAR(delta)) - }}; -} - -// bool_and/bool_or the array and returns a ScalarValue of its corresponding type. -macro_rules! bool_and_or_batch { - ($VALUES:expr, $OP:ident) => {{ - match $VALUES.data_type() { - DataType::Boolean => { - typed_bool_and_or_batch!($VALUES, BooleanArray, Boolean, $OP) - } - e => { - return internal_err!( - "Bool and/Bool or is not expected to receive the type {e:?}" - ); - } - } - }}; -} - -/// dynamically-typed bool_and(array) -> ScalarValue -fn bool_and_batch(values: &ArrayRef) -> Result { - bool_and_or_batch!(values, bool_and) -} - -/// dynamically-typed bool_or(array) -> ScalarValue -fn bool_or_batch(values: &ArrayRef) -> Result { - bool_and_or_batch!(values, bool_or) -} - -/// BOOL_AND aggregate expression -#[derive(Debug, Clone)] -pub struct BoolAnd { - name: String, - pub data_type: DataType, - expr: Arc, - nullable: bool, -} - -impl BoolAnd { - /// Create a new BOOL_AND aggregate function - pub fn new( - expr: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - nullable: true, - } - } -} - -impl AggregateExpr for BoolAnd { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - Ok(Box::::default()) - } - - fn state_fields(&self) -> Result> { - Ok(vec![Field::new( - format_state_name(&self.name, "bool_and"), - self.data_type.clone(), - self.nullable, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { - true - } - - fn create_groups_accumulator(&self) -> Result> { - match self.data_type { - DataType::Boolean => { - Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x && y))) - } - _ => not_impl_err!( - "GroupsAccumulator not supported for {} with {}", - self.name(), - self.data_type - ), - } - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone())) - } - - fn create_sliding_accumulator(&self) -> Result> { - Ok(Box::::default()) - } -} - -impl PartialEq for BoolAnd { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.nullable == x.nullable - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - -#[derive(Debug, Default)] -struct BoolAndAccumulator { - acc: Option, -} - -impl Accumulator for BoolAndAccumulator { - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let values = &values[0]; - self.acc = match (self.acc, bool_and_batch(values)?) { - (None, ScalarValue::Boolean(v)) => v, - (Some(v), ScalarValue::Boolean(None)) => Some(v), - (Some(a), ScalarValue::Boolean(Some(b))) => Some(a && b), - _ => unreachable!(), - }; - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - self.update_batch(states) - } - - fn state(&mut self) -> Result> { - Ok(vec![ScalarValue::Boolean(self.acc)]) - } - - fn evaluate(&mut self) -> Result { - Ok(ScalarValue::Boolean(self.acc)) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - } -} - -/// BOOL_OR aggregate expression -#[derive(Debug, Clone)] -pub struct BoolOr { - name: String, - pub data_type: DataType, - expr: Arc, - nullable: bool, -} - -impl BoolOr { - /// Create a new BOOL_OR aggregate function - pub fn new( - expr: Arc, - name: impl Into, - data_type: DataType, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - nullable: true, - } - } -} - -impl AggregateExpr for BoolOr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.nullable, - )) - } - - fn create_accumulator(&self) -> Result> { - Ok(Box::::default()) - } - - fn state_fields(&self) -> Result> { - Ok(vec![Field::new( - format_state_name(&self.name, "bool_or"), - self.data_type.clone(), - self.nullable, - )]) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { - true - } - - fn create_groups_accumulator(&self) -> Result> { - match self.data_type { - DataType::Boolean => { - Ok(Box::new(BooleanGroupsAccumulator::new(|x, y| x || y))) - } - _ => not_impl_err!( - "GroupsAccumulator not supported for {} with {}", - self.name(), - self.data_type - ), - } - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone())) - } - - fn create_sliding_accumulator(&self) -> Result> { - Ok(Box::::default()) - } -} - -impl PartialEq for BoolOr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.nullable == x.nullable - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - -#[derive(Debug, Default)] -struct BoolOrAccumulator { - acc: Option, -} - -impl Accumulator for BoolOrAccumulator { - fn state(&mut self) -> Result> { - Ok(vec![ScalarValue::Boolean(self.acc)]) - } - - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let values = &values[0]; - self.acc = match (self.acc, bool_or_batch(values)?) { - (None, ScalarValue::Boolean(v)) => v, - (Some(v), ScalarValue::Boolean(None)) => Some(v), - (Some(a), ScalarValue::Boolean(Some(b))) => Some(a || b), - _ => unreachable!(), - }; - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - self.update_batch(states) - } - - fn evaluate(&mut self) -> Result { - Ok(ScalarValue::Boolean(self.acc)) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::col; - use crate::expressions::tests::aggregate; - use crate::generic_test_op; - use arrow::datatypes::*; - use arrow::record_batch::RecordBatch; - - #[test] - fn test_bool_and() -> Result<()> { - let a: ArrayRef = Arc::new(BooleanArray::from(vec![true, true, false])); - generic_test_op!(a, DataType::Boolean, BoolAnd, ScalarValue::from(false)) - } - - #[test] - fn bool_and_with_nulls() -> Result<()> { - let a: ArrayRef = Arc::new(BooleanArray::from(vec![ - Some(true), - None, - Some(true), - Some(true), - ])); - generic_test_op!(a, DataType::Boolean, BoolAnd, ScalarValue::from(true)) - } - - #[test] - fn bool_and_all_nulls() -> Result<()> { - let a: ArrayRef = Arc::new(BooleanArray::from(vec![None, None])); - generic_test_op!(a, DataType::Boolean, BoolAnd, ScalarValue::Boolean(None)) - } - - #[test] - fn test_bool_or() -> Result<()> { - let a: ArrayRef = Arc::new(BooleanArray::from(vec![true, true, false])); - generic_test_op!(a, DataType::Boolean, BoolOr, ScalarValue::from(true)) - } - - #[test] - fn bool_or_with_nulls() -> Result<()> { - let a: ArrayRef = Arc::new(BooleanArray::from(vec![ - Some(false), - None, - Some(false), - Some(false), - ])); - generic_test_op!(a, DataType::Boolean, BoolOr, ScalarValue::from(false)) - } - - #[test] - fn bool_or_all_nulls() -> Result<()> { - let a: ArrayRef = Arc::new(BooleanArray::from(vec![None, None])); - generic_test_op!(a, DataType::Boolean, BoolOr, ScalarValue::Boolean(None)) - } -} diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 1dfe9ffd6905..53cfcfb033a1 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -66,16 +66,6 @@ pub fn create_aggregate_expr( name, data_type, )), - (AggregateFunction::BoolAnd, _) => Arc::new(expressions::BoolAnd::new( - input_phy_exprs[0].clone(), - name, - data_type, - )), - (AggregateFunction::BoolOr, _) => Arc::new(expressions::BoolOr::new( - input_phy_exprs[0].clone(), - name, - data_type, - )), (AggregateFunction::ArrayAgg, false) => { let expr = input_phy_exprs[0].clone(); let nullable = expr.nullable(input_schema)?; @@ -165,9 +155,7 @@ mod tests { use datafusion_common::plan_err; use datafusion_expr::{type_coercion, Signature}; - use crate::expressions::{ - try_cast, ArrayAgg, Avg, BoolAnd, BoolOr, DistinctArrayAgg, Max, Min, - }; + use crate::expressions::{try_cast, ArrayAgg, Avg, DistinctArrayAgg, Max, Min}; use super::*; #[test] @@ -281,48 +269,6 @@ mod tests { Ok(()) } - #[test] - fn test_bool_and_or_expr() -> Result<()> { - let funcs = vec![AggregateFunction::BoolAnd, AggregateFunction::BoolOr]; - let data_types = vec![DataType::Boolean]; - for fun in funcs { - for data_type in &data_types { - let input_schema = - Schema::new(vec![Field::new("c1", data_type.clone(), true)]); - let input_phy_exprs: Vec> = vec![Arc::new( - expressions::Column::new_with_schema("c1", &input_schema).unwrap(), - )]; - let result_agg_phy_exprs = create_physical_agg_expr_for_test( - &fun, - false, - &input_phy_exprs[0..1], - &input_schema, - "c1", - )?; - match fun { - AggregateFunction::BoolAnd => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - assert_eq!( - Field::new("c1", data_type.clone(), true), - result_agg_phy_exprs.field().unwrap() - ); - } - AggregateFunction::BoolOr => { - assert!(result_agg_phy_exprs.as_any().is::()); - assert_eq!("c1", result_agg_phy_exprs.name()); - assert_eq!( - Field::new("c1", data_type.clone(), true), - result_agg_phy_exprs.field().unwrap() - ); - } - _ => {} - }; - } - } - Ok(()) - } - #[test] fn test_sum_avg_expr() -> Result<()> { let funcs = vec![AggregateFunction::Avg]; diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs index a6946e739c97..73d810ec056d 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/mod.rs @@ -25,9 +25,6 @@ pub(crate) mod accumulate { pub use datafusion_physical_expr_common::aggregate::groups_accumulator::accumulate::NullState; -pub(crate) mod bool_op { - pub use datafusion_physical_expr_common::aggregate::groups_accumulator::bool_op::BooleanGroupsAccumulator; -} pub(crate) mod prim_op { pub use datafusion_physical_expr_common::aggregate::groups_accumulator::prim_op::PrimitiveGroupsAccumulator; } diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 87c7deccc2cd..f64c5b1fb260 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -21,7 +21,6 @@ pub(crate) mod array_agg; pub(crate) mod array_agg_distinct; pub(crate) mod array_agg_ordered; pub(crate) mod average; -pub(crate) mod bool_and_or; pub(crate) mod correlation; pub(crate) mod covariance; pub(crate) mod grouping; diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 322610404074..0020aa5f55b2 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -40,7 +40,6 @@ pub use crate::aggregate::array_agg_distinct::DistinctArrayAgg; pub use crate::aggregate::array_agg_ordered::OrderSensitiveArrayAgg; pub use crate::aggregate::average::Avg; pub use crate::aggregate::average::AvgAccumulator; -pub use crate::aggregate::bool_and_or::{BoolAnd, BoolOr}; pub use crate::aggregate::build_in::create_aggregate_expr; pub use crate::aggregate::correlation::Correlation; pub use crate::aggregate::grouping::Grouping; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 6375df721ae6..50356d5b6052 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -494,8 +494,8 @@ enum AggregateFunction { // BIT_AND = 19; // BIT_OR = 20; // BIT_XOR = 21; - BOOL_AND = 22; - BOOL_OR = 23; +// BOOL_AND = 22; +// BOOL_OR = 23; // REGR_SLOPE = 26; // REGR_INTERCEPT = 27; // REGR_COUNT = 28; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 5c483f70d150..8cca0fe4a876 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -538,8 +538,6 @@ impl serde::Serialize for AggregateFunction { Self::ArrayAgg => "ARRAY_AGG", Self::Correlation => "CORRELATION", Self::Grouping => "GROUPING", - Self::BoolAnd => "BOOL_AND", - Self::BoolOr => "BOOL_OR", Self::NthValueAgg => "NTH_VALUE_AGG", }; serializer.serialize_str(variant) @@ -558,8 +556,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "ARRAY_AGG", "CORRELATION", "GROUPING", - "BOOL_AND", - "BOOL_OR", "NTH_VALUE_AGG", ]; @@ -607,8 +603,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "ARRAY_AGG" => Ok(AggregateFunction::ArrayAgg), "CORRELATION" => Ok(AggregateFunction::Correlation), "GROUPING" => Ok(AggregateFunction::Grouping), - "BOOL_AND" => Ok(AggregateFunction::BoolAnd), - "BOOL_OR" => Ok(AggregateFunction::BoolOr), "NTH_VALUE_AGG" => Ok(AggregateFunction::NthValueAgg), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index bc5b6be2ad87..56f14982923d 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1948,8 +1948,8 @@ pub enum AggregateFunction { /// BIT_AND = 19; /// BIT_OR = 20; /// BIT_XOR = 21; - BoolAnd = 22, - BoolOr = 23, + /// BOOL_AND = 22; + /// BOOL_OR = 23; /// REGR_SLOPE = 26; /// REGR_INTERCEPT = 27; /// REGR_COUNT = 28; @@ -1975,8 +1975,6 @@ impl AggregateFunction { AggregateFunction::ArrayAgg => "ARRAY_AGG", AggregateFunction::Correlation => "CORRELATION", AggregateFunction::Grouping => "GROUPING", - AggregateFunction::BoolAnd => "BOOL_AND", - AggregateFunction::BoolOr => "BOOL_OR", AggregateFunction::NthValueAgg => "NTH_VALUE_AGG", } } @@ -1989,8 +1987,6 @@ impl AggregateFunction { "ARRAY_AGG" => Some(Self::ArrayAgg), "CORRELATION" => Some(Self::Correlation), "GROUPING" => Some(Self::Grouping), - "BOOL_AND" => Some(Self::BoolAnd), - "BOOL_OR" => Some(Self::BoolOr), "NTH_VALUE_AGG" => Some(Self::NthValueAgg), _ => None, } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 5bec655bb1ff..ba0e708218cf 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -140,8 +140,6 @@ impl From for AggregateFunction { protobuf::AggregateFunction::Min => Self::Min, protobuf::AggregateFunction::Max => Self::Max, protobuf::AggregateFunction::Avg => Self::Avg, - protobuf::AggregateFunction::BoolAnd => Self::BoolAnd, - protobuf::AggregateFunction::BoolOr => Self::BoolOr, protobuf::AggregateFunction::ArrayAgg => Self::ArrayAgg, protobuf::AggregateFunction::Correlation => Self::Correlation, protobuf::AggregateFunction::Grouping => Self::Grouping, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 66b7c77799ea..08999effa4b1 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -111,8 +111,6 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::Min => Self::Min, AggregateFunction::Max => Self::Max, AggregateFunction::Avg => Self::Avg, - AggregateFunction::BoolAnd => Self::BoolAnd, - AggregateFunction::BoolOr => Self::BoolOr, AggregateFunction::ArrayAgg => Self::ArrayAgg, AggregateFunction::Correlation => Self::Correlation, AggregateFunction::Grouping => Self::Grouping, @@ -376,8 +374,6 @@ pub fn serialize_expr( AggregateFunction::ArrayAgg => protobuf::AggregateFunction::ArrayAgg, AggregateFunction::Min => protobuf::AggregateFunction::Min, AggregateFunction::Max => protobuf::AggregateFunction::Max, - AggregateFunction::BoolAnd => protobuf::AggregateFunction::BoolAnd, - AggregateFunction::BoolOr => protobuf::AggregateFunction::BoolOr, AggregateFunction::Avg => protobuf::AggregateFunction::Avg, AggregateFunction::Correlation => { protobuf::AggregateFunction::Correlation diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index ed966509b842..a9d3736dee08 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -23,10 +23,10 @@ use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ - ArrayAgg, Avg, BinaryExpr, BoolAnd, BoolOr, CaseExpr, CastExpr, Column, Correlation, - CumeDist, DistinctArrayAgg, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, - Max, Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, - OrderSensitiveArrayAgg, Rank, RankType, RowNumber, TryCastExpr, WindowShift, + ArrayAgg, Avg, BinaryExpr, CaseExpr, CastExpr, Column, Correlation, CumeDist, + DistinctArrayAgg, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, Max, Min, + NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, OrderSensitiveArrayAgg, Rank, + RankType, RowNumber, TryCastExpr, WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; @@ -240,10 +240,6 @@ fn aggr_expr_to_aggr_fn(expr: &dyn AggregateExpr) -> Result { let inner = if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::Grouping - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::BoolAnd - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::BoolOr } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::ArrayAgg } else if aggr_expr.downcast_ref::().is_some() { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 61764394ee74..b3966c3f0204 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -59,7 +59,9 @@ use datafusion_expr::{ TryCast, Volatility, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, WindowUDF, WindowUDFImpl, }; -use datafusion_functions_aggregate::expr_fn::{bit_and, bit_or, bit_xor}; +use datafusion_functions_aggregate::expr_fn::{ + bit_and, bit_or, bit_xor, bool_and, bool_or, +}; use datafusion_functions_aggregate::string_agg::string_agg; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, @@ -671,6 +673,8 @@ async fn roundtrip_expr_api() -> Result<()> { bit_or(lit(2)), bit_xor(lit(2)), string_agg(col("a").cast_to(&DataType::Utf8, &schema)?, lit("|")), + bool_and(lit(true)), + bool_or(lit(true)), ]; // ensure expressions created with the expr api can be round tripped From 58d23c5c050f43aa7b867d4f0be7298d8d6cad83 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Thu, 20 Jun 2024 05:59:56 -0700 Subject: [PATCH 33/54] feat: support uint data page extraction (#11018) --- .../physical_plan/parquet/statistics.rs | 42 +++++++++++++++++++ .../core/tests/parquet/arrow_statistics.rs | 8 ++-- 2 files changed, 46 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index 2ca47de990ec..3be060ce6180 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -613,6 +613,48 @@ macro_rules! get_data_page_statistics { ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { paste! { match $data_type { + Some(DataType::UInt8) => Ok(Arc::new( + UInt8Array::from_iter( + [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| u8::try_from(x).ok()) + }) + }) + .flatten() + ) + )), + Some(DataType::UInt16) => Ok(Arc::new( + UInt16Array::from_iter( + [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| u16::try_from(x).ok()) + }) + }) + .flatten() + ) + )), + Some(DataType::UInt32) => Ok(Arc::new( + UInt32Array::from_iter( + [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| u32::try_from(x).ok()) + }) + }) + .flatten() + ))), + Some(DataType::UInt64) => Ok(Arc::new( + UInt64Array::from_iter( + [<$stat_type_prefix Int64DataPageStatsIterator>]::new($iterator) + .map(|x| { + x.into_iter().filter_map(|x| { + x.and_then(|x| u64::try_from(x).ok()) + }) + }) + .flatten() + ))), Some(DataType::Int8) => Ok(Arc::new( Int8Array::from_iter( [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index bdae9f47867e..e48c11fcbaab 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -1352,7 +1352,7 @@ async fn test_uint() { expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u8", - check: Check::RowGroup, + check: Check::Both, } .run(); @@ -1363,7 +1363,7 @@ async fn test_uint() { expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u16", - check: Check::RowGroup, + check: Check::Both, } .run(); @@ -1374,7 +1374,7 @@ async fn test_uint() { expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u32", - check: Check::RowGroup, + check: Check::Both, } .run(); @@ -1385,7 +1385,7 @@ async fn test_uint() { expected_null_counts: UInt64Array::from(vec![0, 0, 0, 0, 0]), expected_row_counts: Some(UInt64Array::from(vec![4, 4, 4, 4, 4])), column_name: "u64", - check: Check::RowGroup, + check: Check::Both, } .run(); } From 5316278ceac6743fde8dfc94880eb63d0c4c3d2e Mon Sep 17 00:00:00 2001 From: Lorrens Pantelis <100197010+LorrensP-2158466@users.noreply.github.com> Date: Thu, 20 Jun 2024 17:39:49 +0200 Subject: [PATCH 34/54] propagate error instead of panicking on out of bounds in physical-expr/src/analysis.rs (#10992) * propogate error instead of panicking * use macro for creating internal df error --- datafusion/physical-expr/src/analysis.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index ca25bfd647b6..e7b199af3743 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -27,7 +27,9 @@ use crate::PhysicalExpr; use arrow::datatypes::Schema; use datafusion_common::stats::Precision; -use datafusion_common::{internal_err, ColumnStatistics, Result, ScalarValue}; +use datafusion_common::{ + internal_datafusion_err, internal_err, ColumnStatistics, Result, ScalarValue, +}; use datafusion_expr::interval_arithmetic::{cardinality_ratio, Interval}; /// The shared context used during the analysis of an expression. Includes @@ -92,7 +94,13 @@ impl ExprBoundaries { col_stats: &ColumnStatistics, col_index: usize, ) -> Result { - let field = &schema.fields()[col_index]; + let field = schema.fields().get(col_index).ok_or_else(|| { + internal_datafusion_err!( + "Could not create `ExprBoundaries`: in `try_from_column` `col_index` + has gone out of bounds with a value of {col_index}, the schema has {} columns.", + schema.fields.len() + ) + })?; let empty_field = ScalarValue::try_from(field.data_type()).unwrap_or(ScalarValue::Null); let interval = Interval::try_new( From 1155b0b15e6ce3a8d5d28e5ecaebf4706448c548 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 20 Jun 2024 11:44:56 -0400 Subject: [PATCH 35/54] Minor: Add more docs and examples for `Transformed` and `TransformedResult` (#11003) --- datafusion/common/src/tree_node.rs | 83 +++++++++++++++++++++++++++++- 1 file changed, 81 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index 276a1cc4c59c..bb54f4e13af9 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -582,7 +582,11 @@ impl TreeNodeRecursion { /// Result of tree walk / transformation APIs /// -/// API users control the transformation by returning: +/// `Transformed` is a wrapper around the tree node data (e.g. `Expr` or +/// `LogicalPlan`). It is used to indicate whether the node was transformed +/// and how the recursion should proceed. +/// +/// [`TreeNode`] API users control the transformation by returning: /// - The resulting (possibly transformed) node, /// - `transformed`: flag indicating whether any change was made to the node /// - `tnr`: [`TreeNodeRecursion`] specifying how to proceed with the recursion. @@ -592,7 +596,66 @@ impl TreeNodeRecursion { /// - `transformed`: flag indicating whether any change was made to the node /// - `tnr`: [`TreeNodeRecursion`] specifying how the recursion ended. /// -/// Example APIs: +/// See also +/// * [`Transformed::update_data`] to modify the node without changing the `transformed` flag +/// * [`Transformed::map_data`] for fallable operation that return the same type +/// * [`Transformed::transform_data`] to chain fallable transformations +/// * [`TransformedResult`] for working with `Result>` +/// +/// # Examples +/// +/// Use [`Transformed::yes`] and [`Transformed::no`] to signal that a node was +/// rewritten and the recursion should continue: +/// +/// ``` +/// # use datafusion_common::tree_node::Transformed; +/// # // note use i64 instead of Expr as Expr is not in datafusion-common +/// # fn orig_expr() -> i64 { 1 } +/// # fn make_new_expr(i: i64) -> i64 { 2 } +/// let expr = orig_expr(); +/// +/// // Create a new `Transformed` object signaling the node was not rewritten +/// let ret = Transformed::no(expr.clone()); +/// assert!(!ret.transformed); +/// +/// // Create a new `Transformed` object signaling the node was rewritten +/// let ret = Transformed::yes(expr); +/// assert!(ret.transformed) +/// ``` +/// +/// Access the node within the `Transformed` object: +/// ``` +/// # use datafusion_common::tree_node::Transformed; +/// # // note use i64 instead of Expr as Expr is not in datafusion-common +/// # fn orig_expr() -> i64 { 1 } +/// # fn make_new_expr(i: i64) -> i64 { 2 } +/// let expr = orig_expr(); +/// +/// // `Transformed` object signaling the node was not rewritten +/// let ret = Transformed::no(expr.clone()); +/// // Access the inner object using .data +/// assert_eq!(expr, ret.data); +/// ``` +/// +/// Transform the node within the `Transformed` object. +/// +/// ``` +/// # use datafusion_common::tree_node::Transformed; +/// # // note use i64 instead of Expr as Expr is not in datafusion-common +/// # fn orig_expr() -> i64 { 1 } +/// # fn make_new_expr(i: i64) -> i64 { 2 } +/// let expr = orig_expr(); +/// let ret = Transformed::no(expr.clone()) +/// .transform_data(|expr| { +/// // closure returns a result and potentially transforms the node +/// // in this example, it does transform the node +/// let new_expr = make_new_expr(expr); +/// Ok(Transformed::yes(new_expr)) +/// }).unwrap(); +/// // transformed flag is the union of the original ans closure's transformed flag +/// assert!(ret.transformed); +/// ``` +/// # Example APIs that use `TreeNode` /// - [`TreeNode`], /// - [`TreeNode::rewrite`], /// - [`TreeNode::transform_down`], @@ -833,6 +896,22 @@ macro_rules! map_until_stop_and_collect { } /// Transformation helper to access [`Transformed`] fields in a [`Result`] easily. +/// +/// # Example +/// Access the internal data of a `Result>` +/// as a `Result` using the `data` method: +/// ``` +/// # use datafusion_common::Result; +/// # use datafusion_common::tree_node::{Transformed, TransformedResult}; +/// # // note use i64 instead of Expr as Expr is not in datafusion-common +/// # fn update_expr() -> i64 { 1 } +/// # fn main() -> Result<()> { +/// let transformed: Result> = Ok(Transformed::yes(update_expr())); +/// // access the internal data of the transformed result, or return the error +/// let transformed_expr = transformed.data()?; +/// # Ok(()) +/// # } +/// ``` pub trait TransformedResult { fn data(self) -> Result; From 18042fd69138e19613844580408a71a200ea6caa Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Thu, 20 Jun 2024 09:58:01 -0700 Subject: [PATCH 36/54] feat: propagate EmptyRelation for more join types (#10963) * feat: propagate empty for more join types * feat: update subquery de-correlation test * tests: simplify tests * refactor: better name * style: clippy * refactor: update tests * refactor: rename * refactor: fix spellings * add slt tests --- .../core/tests/parquet/arrow_statistics.rs | 8 +- .../optimizer/src/eliminate_one_union.rs | 3 +- .../optimizer/src/propagate_empty_relation.rs | 177 +++++++++++++++--- datafusion/optimizer/src/test/mod.rs | 39 +++- datafusion/sqllogictest/test_files/joins.slt | 96 ++++++++++ .../sqllogictest/test_files/subquery.slt | 5 +- 6 files changed, 290 insertions(+), 38 deletions(-) diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index e48c11fcbaab..ddb39fce4076 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! This file contains an end to end test of extracting statitics from parquet files. +//! This file contains an end to end test of extracting statistics from parquet files. //! It writes data into a parquet file, reads statistics and verifies they are correct use std::default::Default; @@ -716,8 +716,8 @@ async fn test_timestamp() { // "seconds_timezoned" --> TimestampSecondArray // "names" --> StringArray // - // The file is created by 4 record batches, each has 5 rowws. - // Since the row group isze is set to 5, those 4 batches will go into 4 row groups + // The file is created by 4 record batches, each has 5 rows. + // Since the row group size is set to 5, those 4 batches will go into 4 row groups // This creates a parquet files of 4 columns named "nanos", "nanos_timezoned", "micros", "micros_timezoned", "millis", "millis_timezoned", "seconds", "seconds_timezoned" let reader = TestReader { scenario: Scenario::Timestamps, @@ -2039,7 +2039,7 @@ async fn test_missing_statistics() { expected_min: Arc::new(Int64Array::from(vec![None])), expected_max: Arc::new(Int64Array::from(vec![None])), expected_null_counts: UInt64Array::from(vec![None]), - expected_row_counts: Some(UInt64Array::from(vec![3])), // stil has row count statistics + expected_row_counts: Some(UInt64Array::from(vec![3])), // still has row count statistics column_name: "i64", check: Check::RowGroup, } diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 68d0ddba8b20..7763e7d3b796 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -88,10 +88,11 @@ mod tests { } fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { - assert_optimized_plan_eq_with_rules( + assert_optimized_plan_with_rules( vec![Arc::new(EliminateOneUnion::new())], plan, expected, + true, ) } diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index d08820c58a05..dfcfc313efcc 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use datafusion_common::tree_node::Transformed; -use datafusion_common::JoinType::Inner; +use datafusion_common::JoinType; use datafusion_common::{internal_err, plan_err, Result}; use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::LogicalPlan; @@ -94,13 +94,9 @@ impl OptimizerRule for PropagateEmptyRelation { Ok(Transformed::no(LogicalPlan::CrossJoin(join.clone()))) } - LogicalPlan::Join(ref join) if join.join_type == Inner => { + LogicalPlan::Join(ref join) => { // TODO: For Join, more join type need to be careful: - // For LeftOuter/LeftSemi/LeftAnti Join, only the left side is empty, the Join result is empty. - // For LeftSemi Join, if the right side is empty, the Join result is empty. // For LeftAnti Join, if the right side is empty, the Join result is left side(should exclude null ??). - // For RightOuter/RightSemi/RightAnti Join, only the right side is empty, the Join result is empty. - // For RightSemi Join, if the left side is empty, the Join result is empty. // For RightAnti Join, if the left side is empty, the Join result is right side(should exclude null ??). // For Full Join, only both sides are empty, the Join result is empty. // For LeftOut/Full Join, if the right side is empty, the Join can be eliminated with a Projection with left side @@ -108,15 +104,52 @@ impl OptimizerRule for PropagateEmptyRelation { // For RightOut/Full Join, if the left side is empty, the Join can be eliminated with a Projection with right side // columns + left side columns replaced with null values. let (left_empty, right_empty) = binary_plan_children_is_empty(&plan)?; - if left_empty || right_empty { - return Ok(Transformed::yes(LogicalPlan::EmptyRelation( - EmptyRelation { + + match join.join_type { + JoinType::Inner if left_empty || right_empty => Ok(Transformed::yes( + LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, schema: join.schema.clone(), - }, - ))); + }), + )), + JoinType::Left if left_empty => Ok(Transformed::yes( + LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: join.schema.clone(), + }), + )), + JoinType::Right if right_empty => Ok(Transformed::yes( + LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: join.schema.clone(), + }), + )), + JoinType::LeftSemi if left_empty || right_empty => Ok( + Transformed::yes(LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: join.schema.clone(), + })), + ), + JoinType::RightSemi if left_empty || right_empty => Ok( + Transformed::yes(LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: join.schema.clone(), + })), + ), + JoinType::LeftAnti if left_empty => Ok(Transformed::yes( + LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: join.schema.clone(), + }), + )), + JoinType::RightAnti if right_empty => Ok(Transformed::yes( + LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: join.schema.clone(), + }), + )), + _ => Ok(Transformed::no(LogicalPlan::Join(join.clone()))), } - Ok(Transformed::no(LogicalPlan::Join(join.clone()))) } LogicalPlan::Aggregate(ref agg) => { if !agg.group_expr.is_empty() { @@ -222,7 +255,7 @@ mod tests { use crate::eliminate_filter::EliminateFilter; use crate::eliminate_nested_union::EliminateNestedUnion; use crate::test::{ - assert_optimized_plan_eq, assert_optimized_plan_eq_with_rules, test_table_scan, + assert_optimized_plan_eq, assert_optimized_plan_with_rules, test_table_scan, test_table_scan_fields, test_table_scan_with_name, }; @@ -232,11 +265,12 @@ mod tests { assert_optimized_plan_eq(Arc::new(PropagateEmptyRelation::new()), plan, expected) } - fn assert_together_optimized_plan_eq( + fn assert_together_optimized_plan( plan: LogicalPlan, expected: &str, + eq: bool, ) -> Result<()> { - assert_optimized_plan_eq_with_rules( + assert_optimized_plan_with_rules( vec![ Arc::new(EliminateFilter::new()), Arc::new(EliminateNestedUnion::new()), @@ -244,6 +278,7 @@ mod tests { ], plan, expected, + eq, ) } @@ -279,7 +314,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } #[test] @@ -292,7 +327,7 @@ mod tests { let plan = LogicalPlanBuilder::from(left).union(right)?.build()?; let expected = "TableScan: test"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } #[test] @@ -317,7 +352,7 @@ mod tests { let expected = "Union\ \n TableScan: test1\ \n TableScan: test4"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } #[test] @@ -342,7 +377,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } #[test] @@ -369,7 +404,7 @@ mod tests { let expected = "Union\ \n TableScan: test2\ \n TableScan: test3"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } #[test] @@ -382,7 +417,7 @@ mod tests { let plan = LogicalPlanBuilder::from(left).union(right)?.build()?; let expected = "TableScan: test"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } #[test] @@ -397,7 +432,103 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) + } + + fn assert_empty_left_empty_right_lp( + left_empty: bool, + right_empty: bool, + join_type: JoinType, + eq: bool, + ) -> Result<()> { + let left_lp = if left_empty { + let left_table_scan = test_table_scan()?; + + LogicalPlanBuilder::from(left_table_scan) + .filter(Expr::Literal(ScalarValue::Boolean(Some(false))))? + .build() + } else { + let scan = test_table_scan_with_name("left").unwrap(); + LogicalPlanBuilder::from(scan).build() + }?; + + let right_lp = if right_empty { + let right_table_scan = test_table_scan_with_name("right")?; + + LogicalPlanBuilder::from(right_table_scan) + .filter(Expr::Literal(ScalarValue::Boolean(Some(false))))? + .build() + } else { + let scan = test_table_scan_with_name("right").unwrap(); + LogicalPlanBuilder::from(scan).build() + }?; + + let plan = LogicalPlanBuilder::from(left_lp) + .join_using( + right_lp, + join_type, + vec![Column::from_name("a".to_string())], + )? + .build()?; + + let expected = "EmptyRelation"; + assert_together_optimized_plan(plan, expected, eq) + } + + #[test] + fn test_join_empty_propagation_rules() -> Result<()> { + // test left join with empty left + assert_empty_left_empty_right_lp(true, false, JoinType::Left, true)?; + + // test right join with empty right + assert_empty_left_empty_right_lp(false, true, JoinType::Right, true)?; + + // test left semi join with empty left + assert_empty_left_empty_right_lp(true, false, JoinType::LeftSemi, true)?; + + // test left semi join with empty right + assert_empty_left_empty_right_lp(false, true, JoinType::LeftSemi, true)?; + + // test right semi join with empty left + assert_empty_left_empty_right_lp(true, false, JoinType::RightSemi, true)?; + + // test right semi join with empty right + assert_empty_left_empty_right_lp(false, true, JoinType::RightSemi, true)?; + + // test left anti join empty left + assert_empty_left_empty_right_lp(true, false, JoinType::LeftAnti, true)?; + + // test right anti join empty right + assert_empty_left_empty_right_lp(false, true, JoinType::RightAnti, true) + } + + #[test] + fn test_join_empty_propagation_rules_noop() -> Result<()> { + // these cases should not result in an empty relation + + // test left join with empty right + assert_empty_left_empty_right_lp(false, true, JoinType::Left, false)?; + + // test right join with empty left + assert_empty_left_empty_right_lp(true, false, JoinType::Right, false)?; + + // test left semi with non-empty left and right + assert_empty_left_empty_right_lp(false, false, JoinType::LeftSemi, false)?; + + // test right semi with non-empty left and right + assert_empty_left_empty_right_lp(false, false, JoinType::RightSemi, false)?; + + // test left anti join with non-empty left and right + assert_empty_left_empty_right_lp(false, false, JoinType::LeftAnti, false)?; + + // test left anti with non-empty left and empty right + assert_empty_left_empty_right_lp(false, true, JoinType::LeftAnti, false)?; + + // test right anti join with non-empty left and right + assert_empty_left_empty_right_lp(false, false, JoinType::RightAnti, false)?; + + // test right anti with empty left and non-empty right + assert_empty_left_empty_right_lp(true, false, JoinType::RightAnti, false) } #[test] @@ -430,6 +561,6 @@ mod tests { let expected = "Projection: a, b, c\ \n TableScan: test"; - assert_together_optimized_plan_eq(plan, expected) + assert_together_optimized_plan(plan, expected, true) } } diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 98d19956df3c..2c7e8644026e 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -121,6 +121,21 @@ pub fn assert_analyzed_plan_eq( Ok(()) } + +pub fn assert_analyzed_plan_ne( + rule: Arc, + plan: LogicalPlan, + expected: &str, +) -> Result<()> { + let options = ConfigOptions::default(); + let analyzed_plan = + Analyzer::with_rules(vec![rule]).execute_and_check(plan, &options, |_, _| {})?; + let formatted_plan = format!("{analyzed_plan:?}"); + assert_ne!(formatted_plan, expected); + + Ok(()) +} + pub fn assert_analyzed_plan_eq_display_indent( rule: Arc, plan: LogicalPlan, @@ -169,21 +184,33 @@ pub fn assert_optimized_plan_eq( Ok(()) } -pub fn assert_optimized_plan_eq_with_rules( +fn generate_optimized_plan_with_rules( rules: Vec>, plan: LogicalPlan, - expected: &str, -) -> Result<()> { +) -> LogicalPlan { fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} let config = &mut OptimizerContext::new() .with_max_passes(1) .with_skip_failing_rules(false); let optimizer = Optimizer::with_rules(rules); - let optimized_plan = optimizer + optimizer .optimize(plan, config, observe) - .expect("failed to optimize plan"); + .expect("failed to optimize plan") +} + +pub fn assert_optimized_plan_with_rules( + rules: Vec>, + plan: LogicalPlan, + expected: &str, + eq: bool, +) -> Result<()> { + let optimized_plan = generate_optimized_plan_with_rules(rules, plan); let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(formatted_plan, expected); + if eq { + assert_eq!(formatted_plan, expected); + } else { + assert_ne!(formatted_plan, expected); + } Ok(()) } diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ea7103255697..0cecd7935a74 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3686,3 +3686,99 @@ set datafusion.explain.logical_plan_only = true; statement ok set datafusion.execution.target_partitions = 2; + +# Inner join with empty left table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 as a WHERE 1=0 +) AS a INNER JOIN (SELECT 1 as a) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Inner join with empty right table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a +) AS a INNER JOIN (SELECT 1 AS a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Left join with empty left table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 as a WHERE 1=0 +) AS a LEFT JOIN (SELECT 1 as a) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Left join with empty left and empty right table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 as a WHERE 1=0 +) AS a LEFT JOIN (SELECT 1 as a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Right join with empty right table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a +) AS a RIGHT JOIN (SELECT 1 AS a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Right join with empty right and empty left table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 as a WHERE 1=0 +) AS a RIGHT JOIN (SELECT 1 as a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Left SEMI join with empty left table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a +) AS a LEFT SEMI JOIN (SELECT 1 AS a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Left SEMI join with empty right table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a WHERE 1=0 +) AS a LEFT SEMI JOIN (SELECT 1 AS a) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Right SEMI join with empty left table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a WHERE 1=0 +) AS a RIGHT SEMI JOIN (SELECT 1 AS a) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Right SEMI join with empty right table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a +) AS a RIGHT SEMI JOIN (SELECT 1 AS a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Left ANTI join with empty left table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a WHERE 1=0 +) AS a LEFT ANTI JOIN (SELECT 1 AS a) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation + +# Right ANTI join with empty right table +query TT +EXPLAIN SELECT * FROM ( + SELECT 1 AS a +) AS a RIGHT ANTI JOIN (SELECT 1 AS a WHERE 1=0) AS b ON a.a=b.a; +---- +logical_plan EmptyRelation diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index f325d5567603..dbdb7fc76b8b 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -626,10 +626,7 @@ SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 0) ---- -logical_plan -01)LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id -02)--TableScan: t1 projection=[t1_id, t1_name] -03)--EmptyRelation +logical_plan EmptyRelation query IT rowsort SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 0) From 1f3ba116a428c932fc34b7cbfbecd1b8fa1d60ab Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 22 Jun 2024 00:13:53 +0800 Subject: [PATCH 37/54] doc: Update links in the documantation (#11044) --- datafusion/optimizer/src/analyzer/mod.rs | 2 +- datafusion/optimizer/src/optimizer.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/optimizer/src/analyzer/mod.rs b/datafusion/optimizer/src/analyzer/mod.rs index 121e46cc950f..32bb2bc70452 100644 --- a/datafusion/optimizer/src/analyzer/mod.rs +++ b/datafusion/optimizer/src/analyzer/mod.rs @@ -57,7 +57,7 @@ pub mod type_coercion; /// Use [`SessionState::add_analyzer_rule`] to register additional /// `AnalyzerRule`s. /// -/// [`SessionState::add_analyzer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionState.html#method.add_analyzer_rule +/// [`SessionState::add_analyzer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_analyzer_rule pub trait AnalyzerRule { /// Rewrite `plan` fn analyze(&self, plan: LogicalPlan, config: &ConfigOptions) -> Result; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 998eeb7167ee..75ebc92ece33 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -68,7 +68,7 @@ use crate::utils::log_plan; /// `OptimizerRule`s. /// /// [`AnalyzerRule`]: crate::analyzer::AnalyzerRule -/// [`SessionState::add_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionState.html#method.add_optimizer_rule +/// [`SessionState::add_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_optimizer_rule pub trait OptimizerRule { /// Try and rewrite `plan` to an optimized form, returning None if the plan From 5498a02853021fc07a57743990d207e25ada27f4 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Fri, 21 Jun 2024 12:14:57 -0400 Subject: [PATCH 38/54] Add drop_columns to dataframe api (#11010) * Add drop_columns to dataframe api #11007 * Prettier cleanup * Added additional drop_columns tests and fixed issue with nonexistent columns. --- datafusion/core/src/dataframe/mod.rs | 169 +++++++++++++++++++++++++++ docs/source/user-guide/dataframe.md | 1 + 2 files changed, 170 insertions(+) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index b5c58eff577c..43ac3992be78 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -244,6 +244,42 @@ impl DataFrame { }) } + /// Returns a new DataFrame containing all columns except the specified columns. + /// + /// ``` + /// # use datafusion::prelude::*; + /// # use datafusion::error::Result; + /// # #[tokio::main] + /// # async fn main() -> Result<()> { + /// let ctx = SessionContext::new(); + /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let df = df.drop_columns(&["a"])?; + /// # Ok(()) + /// # } + /// ``` + pub fn drop_columns(self, columns: &[&str]) -> Result { + let fields_to_drop = columns + .iter() + .map(|name| { + self.plan + .schema() + .qualified_field_with_unqualified_name(name) + }) + .filter(|r| r.is_ok()) + .collect::>>()?; + let expr: Vec = self + .plan + .schema() + .fields() + .into_iter() + .enumerate() + .map(|(idx, _)| self.plan.schema().qualified_field(idx)) + .filter(|(qualifier, f)| !fields_to_drop.contains(&(*qualifier, f))) + .map(|(qualifier, field)| Expr::Column(Column::from((qualifier, field)))) + .collect(); + self.select(expr) + } + /// Expand each list element of a column to multiple rows. #[deprecated(since = "37.0.0", note = "use unnest_columns instead")] pub fn unnest_column(self, column: &str) -> Result { @@ -1799,6 +1835,139 @@ mod tests { Ok(()) } + #[tokio::test] + async fn drop_columns() -> Result<()> { + // build plan using Table API + let t = test_table().await?; + let t2 = t.drop_columns(&["c2", "c11"])?; + let plan = t2.plan.clone(); + + // build query using SQL + let sql_plan = create_plan( + "SELECT c1,c3,c4,c5,c6,c7,c8,c9,c10,c12,c13 FROM aggregate_test_100", + ) + .await?; + + // the two plans should be identical + assert_same_plan(&plan, &sql_plan); + + Ok(()) + } + + #[tokio::test] + async fn drop_columns_with_duplicates() -> Result<()> { + // build plan using Table API + let t = test_table().await?; + let t2 = t.drop_columns(&["c2", "c11", "c2", "c2"])?; + let plan = t2.plan.clone(); + + // build query using SQL + let sql_plan = create_plan( + "SELECT c1,c3,c4,c5,c6,c7,c8,c9,c10,c12,c13 FROM aggregate_test_100", + ) + .await?; + + // the two plans should be identical + assert_same_plan(&plan, &sql_plan); + + Ok(()) + } + + #[tokio::test] + async fn drop_columns_with_nonexistent_columns() -> Result<()> { + // build plan using Table API + let t = test_table().await?; + let t2 = t.drop_columns(&["canada", "c2", "rocks"])?; + let plan = t2.plan.clone(); + + // build query using SQL + let sql_plan = create_plan( + "SELECT c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13 FROM aggregate_test_100", + ) + .await?; + + // the two plans should be identical + assert_same_plan(&plan, &sql_plan); + + Ok(()) + } + + #[tokio::test] + async fn drop_columns_with_empty_array() -> Result<()> { + // build plan using Table API + let t = test_table().await?; + let t2 = t.drop_columns(&[])?; + let plan = t2.plan.clone(); + + // build query using SQL + let sql_plan = create_plan( + "SELECT c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13 FROM aggregate_test_100", + ) + .await?; + + // the two plans should be identical + assert_same_plan(&plan, &sql_plan); + + Ok(()) + } + + #[tokio::test] + async fn drop_with_quotes() -> Result<()> { + // define data with a column name that has a "." in it: + let array1: Int32Array = [1, 10].into_iter().collect(); + let array2: Int32Array = [2, 11].into_iter().collect(); + let batch = RecordBatch::try_from_iter(vec![ + ("f\"c1", Arc::new(array1) as _), + ("f\"c2", Arc::new(array2) as _), + ])?; + + let ctx = SessionContext::new(); + ctx.register_batch("t", batch)?; + + let df = ctx.table("t").await?.drop_columns(&["f\"c1"])?; + + let df_results = df.collect().await?; + + assert_batches_sorted_eq!( + [ + "+------+", + "| f\"c2 |", + "+------+", + "| 2 |", + "| 11 |", + "+------+" + ], + &df_results + ); + + Ok(()) + } + + #[tokio::test] + async fn drop_with_periods() -> Result<()> { + // define data with a column name that has a "." in it: + let array1: Int32Array = [1, 10].into_iter().collect(); + let array2: Int32Array = [2, 11].into_iter().collect(); + let batch = RecordBatch::try_from_iter(vec![ + ("f.c1", Arc::new(array1) as _), + ("f.c2", Arc::new(array2) as _), + ])?; + + let ctx = SessionContext::new(); + ctx.register_batch("t", batch)?; + + let df = ctx.table("t").await?.drop_columns(&["f.c1"])?; + + let df_results = df.collect().await?; + + assert_batches_sorted_eq!( + ["+------+", "| f.c2 |", "+------+", "| 2 |", "| 11 |", "+------+"], + &df_results + ); + + Ok(()) + } + #[tokio::test] async fn aggregate() -> Result<()> { // build plan using DataFrame API diff --git a/docs/source/user-guide/dataframe.md b/docs/source/user-guide/dataframe.md index c0210200a246..744a719e77be 100644 --- a/docs/source/user-guide/dataframe.md +++ b/docs/source/user-guide/dataframe.md @@ -64,6 +64,7 @@ execution. The plan is evaluated (executed) when an action method is invoked, su | ------------------- | ------------------------------------------------------------------------------------------------------------------------------------------ | | aggregate | Perform an aggregate query with optional grouping expressions. | | distinct | Filter out duplicate rows. | +| drop_columns | Create a projection with all but the provided column names. | | except | Calculate the exception of two DataFrames. The two DataFrames must have exactly the same schema | | filter | Filter a DataFrame to only include rows that match the specified filter expression. | | intersect | Calculate the intersection of two DataFrames. The two DataFrames must have exactly the same schema | From fd5a68f8021b878d1f43e1451feb78154d9378d9 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 22 Jun 2024 00:17:48 +0800 Subject: [PATCH 39/54] Push down filter plan for non-unnest column (#11019) * push down non-unnest only Signed-off-by: jayzhan211 * add doc Signed-off-by: jayzhan211 * add doc Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * rewrite unnest push donw filter Signed-off-by: jayzhan211 * remove comment Signed-off-by: jayzhan211 * avoid double recurisve Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/expr/src/logical_plan/plan.rs | 9 ++ datafusion/optimizer/src/push_down_filter.rs | 84 +++++++++++-------- .../test_files/push_down_filter.slt | 23 ++++- 3 files changed, 80 insertions(+), 36 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 85958223ac97..6e7efaf39e3e 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -281,6 +281,15 @@ pub enum LogicalPlan { RecursiveQuery(RecursiveQuery), } +impl Default for LogicalPlan { + fn default() -> Self { + LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: Arc::new(DFSchema::empty()), + }) + } +} + impl LogicalPlan { /// Get a reference to the logical plan's schema pub fn schema(&self) -> &DFSchemaRef { diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 295039af2f19..68339a84649d 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -693,8 +693,9 @@ impl OptimizerRule for PushDownFilter { insert_below(LogicalPlan::SubqueryAlias(subquery_alias), new_filter) } LogicalPlan::Projection(projection) => { + let predicates = split_conjunction_owned(filter.predicate.clone()); let (new_projection, keep_predicate) = - rewrite_projection(filter.predicate.clone(), projection)?; + rewrite_projection(predicates, projection)?; if new_projection.transformed { match keep_predicate { None => Ok(new_projection), @@ -709,41 +710,54 @@ impl OptimizerRule for PushDownFilter { } } LogicalPlan::Unnest(mut unnest) => { - // collect all the Expr::Column in predicate recursively - let mut accum: HashSet = HashSet::new(); - expr_to_columns(&filter.predicate, &mut accum)?; + let predicates = split_conjunction_owned(filter.predicate.clone()); + let mut non_unnest_predicates = vec![]; + let mut unnest_predicates = vec![]; + for predicate in predicates { + // collect all the Expr::Column in predicate recursively + let mut accum: HashSet = HashSet::new(); + expr_to_columns(&predicate, &mut accum)?; + + if unnest.exec_columns.iter().any(|c| accum.contains(c)) { + unnest_predicates.push(predicate); + } else { + non_unnest_predicates.push(predicate); + } + } - if unnest.exec_columns.iter().any(|c| accum.contains(c)) { + // Unnest predicates should not be pushed down. + // If no non-unnest predicates exist, early return + if non_unnest_predicates.is_empty() { filter.input = Arc::new(LogicalPlan::Unnest(unnest)); return Ok(Transformed::no(LogicalPlan::Filter(filter))); } - // Unnest is built above Projection, so we only take Projection into consideration - match unwrap_arc(unnest.input) { - LogicalPlan::Projection(projection) => { - let (new_projection, keep_predicate) = - rewrite_projection(filter.predicate.clone(), projection)?; - unnest.input = Arc::new(new_projection.data); - - if new_projection.transformed { - match keep_predicate { - None => Ok(Transformed::yes(LogicalPlan::Unnest(unnest))), - Some(keep_predicate) => Ok(Transformed::yes( - LogicalPlan::Filter(Filter::try_new( - keep_predicate, - Arc::new(LogicalPlan::Unnest(unnest)), - )?), - )), - } - } else { - filter.input = Arc::new(LogicalPlan::Unnest(unnest)); - Ok(Transformed::no(LogicalPlan::Filter(filter))) - } - } - child => { - filter.input = Arc::new(child); - Ok(Transformed::no(LogicalPlan::Filter(filter))) - } + // Push down non-unnest filter predicate + // Unnest + // Unenst Input (Projection) + // -> rewritten to + // Unnest + // Filter + // Unenst Input (Projection) + + let unnest_input = std::mem::take(&mut unnest.input); + + let filter_with_unnest_input = LogicalPlan::Filter(Filter::try_new( + conjunction(non_unnest_predicates).unwrap(), // Safe to unwrap since non_unnest_predicates is not empty. + unnest_input, + )?); + + // Directly assign new filter plan as the new unnest's input. + // The new filter plan will go through another rewrite pass since the rule itself + // is applied recursively to all the child from top to down + let unnest_plan = + insert_below(LogicalPlan::Unnest(unnest), filter_with_unnest_input)?; + + match conjunction(unnest_predicates) { + None => Ok(unnest_plan), + Some(predicate) => Ok(Transformed::yes(LogicalPlan::Filter( + Filter::try_new(predicate, Arc::new(unnest_plan.data))?, + ))), } } LogicalPlan::Union(ref union) => { @@ -958,6 +972,10 @@ impl OptimizerRule for PushDownFilter { /// `plan` is a LogicalPlan for `projection` with possibly a new FilterExec below it. /// `remaining_predicate` is any part of the predicate that could not be pushed down /// +/// # Args +/// - predicates: Split predicates like `[foo=5, bar=6]` +/// - projection: The target projection plan to push down the predicates +/// /// # Example /// /// Pushing a predicate like `foo=5 AND bar=6` with an input plan like this: @@ -974,7 +992,7 @@ impl OptimizerRule for PushDownFilter { /// ... /// ``` fn rewrite_projection( - predicate: Expr, + predicates: Vec, projection: Projection, ) -> Result<(Transformed, Option)> { // A projection is filter-commutable if it do not contain volatile predicates or contain volatile @@ -994,7 +1012,7 @@ fn rewrite_projection( let mut push_predicates = vec![]; let mut keep_predicates = vec![]; - for expr in split_conjunction_owned(predicate) { + for expr in predicates { if contain(&expr, &volatile_map) { keep_predicates.push(expr); } else { diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 5029ab170a18..3ca187ddee84 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -67,17 +67,34 @@ select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where 5 2 # Could push the filter (column1 = 2) down below unnest -# https://github.com/apache/datafusion/issues/11016 query TT explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; ---- logical_plan 01)Projection: unnest(v.column2) AS uc2, v.column1 -02)--Filter: unnest(v.column2) > Int64(3) AND v.column1 = Int64(2) +02)--Filter: unnest(v.column2) > Int64(3) 03)----Unnest: lists[unnest(v.column2)] structs[] 04)------Projection: v.column2 AS unnest(v.column2), v.column1 -05)--------TableScan: v projection=[column1, column2] +05)--------Filter: v.column1 = Int64(2) +06)----------TableScan: v projection=[column1, column2] + +query II +select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 OR column1 = 2; +---- +3 2 +4 2 +5 2 +# only non-unnest filter in AND clause could be pushed down +query TT +explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 OR column1 = 2; +---- +logical_plan +01)Projection: unnest(v.column2) AS uc2, v.column1 +02)--Filter: unnest(v.column2) > Int64(3) OR v.column1 = Int64(2) +03)----Unnest: lists[unnest(v.column2)] structs[] +04)------Projection: v.column2 AS unnest(v.column2), v.column1 +05)--------TableScan: v projection=[column1, column2] statement ok drop table v; From 4a0c7f35a02ef71fcea518a3566edc9b56866cae Mon Sep 17 00:00:00 2001 From: Marvin Lanhenke <62298609+marvinlanhenke@users.noreply.github.com> Date: Fri, 21 Jun 2024 18:18:14 +0200 Subject: [PATCH 40/54] Consider timezones with `UTC` and `+00:00` to be the same (#10960) * feat: add temporal_coercion check * fix: add return stmt * chore: add slts * fix: remove println * Update datafusion/expr/src/type_coercion/binary.rs --------- Co-authored-by: Andrew Lamb --- datafusion/expr/src/type_coercion/binary.rs | 14 +++++++---- .../sqllogictest/test_files/timestamps.slt | 23 +++++++++++++++++++ 2 files changed, 32 insertions(+), 5 deletions(-) diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 615bb3ac568c..ea9d0c2fe72e 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -1050,12 +1050,16 @@ fn temporal_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { let tz = match (lhs_tz, rhs_tz) { - // can't cast across timezones (Some(lhs_tz), Some(rhs_tz)) => { - if lhs_tz != rhs_tz { - return None; - } else { - Some(lhs_tz.clone()) + match (lhs_tz.as_ref(), rhs_tz.as_ref()) { + // UTC and "+00:00" are the same by definition. Most other timezones + // do not have a 1-1 mapping between timezone and an offset from UTC + ("UTC", "+00:00") | ("+00:00", "UTC") => Some(lhs_tz.clone()), + (lhs, rhs) if lhs == rhs => Some(lhs_tz.clone()), + // can't cast across timezones + _ => { + return None; + } } } (Some(lhs_tz), None) => Some(lhs_tz.clone()), diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 7d5d601bbfdd..96d846d449e1 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2801,3 +2801,26 @@ query B select current_time = current_time; ---- true + +# Test temporal coercion for UTC +query ? +select arrow_cast('2024-06-17T11:00:00', 'Timestamp(Nanosecond, Some("UTC"))') - arrow_cast('2024-06-17T12:00:00', 'Timestamp(Microsecond, Some("UTC"))'); +---- +0 days -1 hours 0 mins 0.000000 secs + +query ? +select arrow_cast('2024-06-17T13:00:00', 'Timestamp(Nanosecond, Some("+00:00"))') - arrow_cast('2024-06-17T12:00:00', 'Timestamp(Microsecond, Some("UTC"))'); +---- +0 days 1 hours 0 mins 0.000000 secs + +query ? +select arrow_cast('2024-06-17T13:00:00', 'Timestamp(Nanosecond, Some("UTC"))') - arrow_cast('2024-06-17T12:00:00', 'Timestamp(Microsecond, Some("+00:00"))'); +---- +0 days 1 hours 0 mins 0.000000 secs + +# not supported: coercion across timezones +query error +select arrow_cast('2024-06-17T13:00:00', 'Timestamp(Nanosecond, Some("UTC"))') - arrow_cast('2024-06-17T12:00:00', 'Timestamp(Microsecond, Some("+01:00"))'); + +query error +select arrow_cast('2024-06-17T13:00:00', 'Timestamp(Nanosecond, Some("+00:00"))') - arrow_cast('2024-06-17T12:00:00', 'Timestamp(Microsecond, Some("+01:00"))'); From 6dffc53e76fb9a785dafd55106dd4033e744bf8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Sat, 22 Jun 2024 00:20:21 +0800 Subject: [PATCH 41/54] Deprecate `OptimizerRule::try_optimize` (#11022) * Deprecate OptimizerRule::try_optimize * optimize_children * Apply review suggestions * Fix clippy lint --- datafusion-examples/examples/rewrite_expr.rs | 48 ++++++++++--------- .../tests/user_defined/user_defined_plan.rs | 44 +++++++++++------ datafusion/optimizer/src/lib.rs | 1 + datafusion/optimizer/src/optimizer.rs | 21 +++++++- datafusion/optimizer/src/utils.rs | 17 +++++-- 5 files changed, 89 insertions(+), 42 deletions(-) diff --git a/datafusion-examples/examples/rewrite_expr.rs b/datafusion-examples/examples/rewrite_expr.rs index d8965888eab6..556687a46ab4 100644 --- a/datafusion-examples/examples/rewrite_expr.rs +++ b/datafusion-examples/examples/rewrite_expr.rs @@ -18,13 +18,13 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{plan_err, Result, ScalarValue}; +use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ AggregateUDF, Between, Expr, Filter, LogicalPlan, ScalarUDF, TableSource, WindowUDF, }; use datafusion_optimizer::analyzer::{Analyzer, AnalyzerRule}; -use datafusion_optimizer::optimizer::Optimizer; -use datafusion_optimizer::{utils, OptimizerConfig, OptimizerContext, OptimizerRule}; +use datafusion_optimizer::optimizer::{ApplyOrder, Optimizer}; +use datafusion_optimizer::{OptimizerConfig, OptimizerContext, OptimizerRule}; use datafusion_sql::planner::{ContextProvider, SqlToRel}; use datafusion_sql::sqlparser::dialect::PostgreSqlDialect; use datafusion_sql::sqlparser::parser::Parser; @@ -133,30 +133,34 @@ impl OptimizerRule for MyOptimizerRule { fn try_optimize( &self, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, + _plan: &LogicalPlan, + _config: &dyn OptimizerConfig, ) -> Result> { - // recurse down and optimize children first - let optimized_plan = utils::optimize_children(self, plan, config)?; - match optimized_plan { - Some(LogicalPlan::Filter(filter)) => { + unreachable!() + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::BottomUp) + } + + fn supports_rewrite(&self) -> bool { + true + } + + fn rewrite( + &self, + plan: LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result, DataFusionError> { + match plan { + LogicalPlan::Filter(filter) => { let predicate = my_rewrite(filter.predicate.clone())?; - Ok(Some(LogicalPlan::Filter(Filter::try_new( + Ok(Transformed::yes(LogicalPlan::Filter(Filter::try_new( predicate, - filter.input, + filter.input.clone(), )?))) } - Some(optimized_plan) => Ok(Some(optimized_plan)), - None => match plan { - LogicalPlan::Filter(filter) => { - let predicate = my_rewrite(filter.predicate.clone())?; - Ok(Some(LogicalPlan::Filter(Filter::try_new( - predicate, - filter.input.clone(), - )?))) - } - _ => Ok(None), - }, + _ => Ok(Transformed::no(plan)), } } } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 07622e48afaf..ebf907c5e2c0 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -80,7 +80,7 @@ use datafusion::{ Expr, Extension, Limit, LogicalPlan, Sort, UserDefinedLogicalNode, UserDefinedLogicalNodeCore, }, - optimizer::{optimize_children, OptimizerConfig, OptimizerRule}, + optimizer::{OptimizerConfig, OptimizerRule}, physical_expr::EquivalenceProperties, physical_plan::{ DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, @@ -92,6 +92,8 @@ use datafusion::{ }; use async_trait::async_trait; +use datafusion_common::tree_node::Transformed; +use datafusion_optimizer::optimizer::ApplyOrder; use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches @@ -282,9 +284,29 @@ impl OptimizerRule for TopKOptimizerRule { // Example rewrite pass to insert a user defined LogicalPlanNode fn try_optimize( &self, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, + _plan: &LogicalPlan, + _config: &dyn OptimizerConfig, ) -> Result> { + unreachable!() + } + + fn name(&self) -> &str { + "topk" + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::TopDown) + } + + fn supports_rewrite(&self) -> bool { + true + } + + fn rewrite( + &self, + plan: LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result, DataFusionError> { // Note: this code simply looks for the pattern of a Limit followed by a // Sort and replaces it by a TopK node. It does not handle many // edge cases (e.g multiple sort columns, sort ASC / DESC), etc. @@ -292,7 +314,7 @@ impl OptimizerRule for TopKOptimizerRule { fetch: Some(fetch), input, .. - }) = plan + }) = &plan { if let LogicalPlan::Sort(Sort { ref expr, @@ -302,12 +324,10 @@ impl OptimizerRule for TopKOptimizerRule { { if expr.len() == 1 { // we found a sort with a single sort expr, replace with a a TopK - return Ok(Some(LogicalPlan::Extension(Extension { + return Ok(Transformed::yes(LogicalPlan::Extension(Extension { node: Arc::new(TopKPlanNode { k: *fetch, - input: self - .try_optimize(input.as_ref(), config)? - .unwrap_or_else(|| input.as_ref().clone()), + input: input.as_ref().clone(), expr: expr[0].clone(), }), }))); @@ -315,13 +335,7 @@ impl OptimizerRule for TopKOptimizerRule { } } - // If we didn't find the Limit/Sort combination, recurse as - // normal and build the result. - optimize_children(self, plan, config) - } - - fn name(&self) -> &str { - "topk" + Ok(Transformed::no(plan)) } } diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index c172d5979756..a6a9e5cf26ea 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -61,6 +61,7 @@ pub mod test; pub use analyzer::{Analyzer, AnalyzerRule}; pub use optimizer::{Optimizer, OptimizerConfig, OptimizerContext, OptimizerRule}; +#[allow(deprecated)] pub use utils::optimize_children; pub(crate) mod join_key_set; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 75ebc92ece33..7a3ea6ed4cc1 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -77,6 +77,10 @@ pub trait OptimizerRule { /// Note this API will be deprecated in the future as it requires `clone`ing /// the input plan, which can be expensive. OptimizerRules should implement /// [`Self::rewrite`] instead. + #[deprecated( + since = "40.0.0", + note = "please implement supports_rewrite and rewrite instead" + )] fn try_optimize( &self, plan: &LogicalPlan, @@ -332,6 +336,7 @@ fn optimize_plan_node( return rule.rewrite(plan, config); } + #[allow(deprecated)] rule.try_optimize(&plan, config).map(|maybe_plan| { match maybe_plan { Some(new_plan) => { @@ -483,7 +488,7 @@ mod tests { use std::sync::{Arc, Mutex}; use datafusion_common::tree_node::Transformed; - use datafusion_common::{plan_err, DFSchema, DFSchemaRef, Result}; + use datafusion_common::{plan_err, DFSchema, DFSchemaRef, DataFusionError, Result}; use datafusion_expr::logical_plan::EmptyRelation; use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, Projection}; @@ -667,12 +672,24 @@ mod tests { _: &LogicalPlan, _: &dyn OptimizerConfig, ) -> Result> { - plan_err!("rule failed") + unreachable!() } fn name(&self) -> &str { "bad rule" } + + fn supports_rewrite(&self) -> bool { + true + } + + fn rewrite( + &self, + _plan: LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result, DataFusionError> { + plan_err!("rule failed") + } } /// Replaces whatever plan with a single table scan diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index fd47cb23b108..6218140409b5 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -35,6 +35,10 @@ use log::{debug, trace}; /// This also handles the case when the `plan` is a [`LogicalPlan::Explain`]. /// /// Returning `Ok(None)` indicates that the plan can't be optimized by the `optimizer`. +#[deprecated( + since = "40.0.0", + note = "please use OptimizerRule::apply_order with ApplyOrder::BottomUp instead" +)] pub fn optimize_children( optimizer: &impl OptimizerRule, plan: &LogicalPlan, @@ -43,9 +47,16 @@ pub fn optimize_children( let mut new_inputs = Vec::with_capacity(plan.inputs().len()); let mut plan_is_changed = false; for input in plan.inputs() { - let new_input = optimizer.try_optimize(input, config)?; - plan_is_changed = plan_is_changed || new_input.is_some(); - new_inputs.push(new_input.unwrap_or_else(|| input.clone())) + if optimizer.supports_rewrite() { + let new_input = optimizer.rewrite(input.clone(), config)?; + plan_is_changed = plan_is_changed || new_input.transformed; + new_inputs.push(new_input.data); + } else { + #[allow(deprecated)] + let new_input = optimizer.try_optimize(input, config)?; + plan_is_changed = plan_is_changed || new_input.is_some(); + new_inputs.push(new_input.unwrap_or_else(|| input.clone())) + } } if plan_is_changed { let exprs = plan.expressions(); From 098ba30ce5a950f51f56d43a4cdcc192af2bf9e1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 21 Jun 2024 19:25:40 +0300 Subject: [PATCH 42/54] Relax combine partial final rule (#10913) * Minor changes * Minor changes * Re-introduce group by expression check --- .../combine_partial_final_agg.rs | 66 +++++------------ .../sqllogictest/test_files/group_by.slt | 71 +++++++++++++++++++ datafusion/sqllogictest/test_files/joins.slt | 23 +++--- 3 files changed, 99 insertions(+), 61 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 b57f36f728d7..940b09131a77 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -27,8 +27,7 @@ use crate::physical_plan::ExecutionPlan; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; +use datafusion_physical_expr::{physical_exprs_equal, AggregateExpr, PhysicalExpr}; /// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs /// into a Single AggregateExec if their grouping exprs and aggregate exprs equal. @@ -132,19 +131,23 @@ type GroupExprsRef<'a> = ( &'a [Option>], ); -type GroupExprs = ( - PhysicalGroupBy, - Vec>, - Vec>>, -); - fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { - let (final_group_by, final_aggr_expr, final_filter_expr) = - normalize_group_exprs(final_agg); - let (input_group_by, input_aggr_expr, input_filter_expr) = - normalize_group_exprs(partial_agg); - - final_group_by.eq(&input_group_by) + let (final_group_by, final_aggr_expr, final_filter_expr) = final_agg; + let (input_group_by, input_aggr_expr, input_filter_expr) = partial_agg; + + // Compare output expressions of the partial, and input expressions of the final operator. + physical_exprs_equal( + &input_group_by.output_exprs(), + &final_group_by.input_exprs(), + ) && input_group_by.groups() == final_group_by.groups() + && input_group_by.null_expr().len() == final_group_by.null_expr().len() + && input_group_by + .null_expr() + .iter() + .zip(final_group_by.null_expr().iter()) + .all(|((lhs_expr, lhs_str), (rhs_expr, rhs_str))| { + lhs_expr.eq(rhs_expr) && lhs_str == rhs_str + }) && final_aggr_expr.len() == input_aggr_expr.len() && final_aggr_expr .iter() @@ -160,41 +163,6 @@ fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { ) } -// To compare the group expressions between the final and partial aggregations, need to discard all the column indexes and compare -fn normalize_group_exprs(group_exprs: GroupExprsRef) -> GroupExprs { - let (group, agg, filter) = group_exprs; - let new_group_expr = group - .expr() - .iter() - .map(|(expr, name)| (discard_column_index(expr.clone()), name.clone())) - .collect::>(); - let new_group = PhysicalGroupBy::new( - new_group_expr, - group.null_expr().to_vec(), - group.groups().to_vec(), - ); - (new_group, agg.to_vec(), filter.to_vec()) -} - -fn discard_column_index(group_expr: Arc) -> Arc { - group_expr - .clone() - .transform(|expr| { - let normalized_form: Option> = - match expr.as_any().downcast_ref::() { - Some(column) => Some(Arc::new(Column::new(column.name(), 0))), - None => None, - }; - Ok(if let Some(normalized_form) = normalized_form { - Transformed::yes(normalized_form) - } else { - Transformed::no(expr) - }) - }) - .data() - .unwrap_or(group_expr) -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 8ccf3ae85345..fff3977fe1e6 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -5064,3 +5064,74 @@ statement error DataFusion error: Error during planning: Cannot find column with SELECT a, b, COUNT(1) FROM multiple_ordered_table GROUP BY 1, 2, 4, 5, 6; + +statement ok +set datafusion.execution.target_partitions = 1; + +# Create a table that contains various keywords, with their corresponding timestamps +statement ok +CREATE TABLE keywords_stream ( + ts TIMESTAMP, + sn INTEGER PRIMARY KEY, + keyword VARCHAR NOT NULL +); + +statement ok +INSERT INTO keywords_stream(ts, sn, keyword) VALUES +('2024-01-01T00:00:00Z', '0', 'Drug'), +('2024-01-01T00:00:05Z', '1', 'Bomb'), +('2024-01-01T00:00:10Z', '2', 'Theft'), +('2024-01-01T00:00:15Z', '3', 'Gun'), +('2024-01-01T00:00:20Z', '4', 'Calm'); + +# Create a table that contains alert keywords +statement ok +CREATE TABLE ALERT_KEYWORDS(keyword VARCHAR NOT NULL); + +statement ok +INSERT INTO ALERT_KEYWORDS VALUES +('Drug'), +('Bomb'), +('Theft'), +('Gun'), +('Knife'), +('Fire'); + +query TT +explain SELECT + DATE_BIN(INTERVAL '2' MINUTE, ts, '2000-01-01') AS ts_chunk, + COUNT(keyword) AS alert_keyword_count +FROM + keywords_stream +WHERE + keywords_stream.keyword IN (SELECT keyword FROM ALERT_KEYWORDS) +GROUP BY + ts_chunk; +---- +logical_plan +01)Projection: date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01")) AS ts_chunk, COUNT(keywords_stream.keyword) AS alert_keyword_count +02)--Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"), keywords_stream.ts, TimestampNanosecond(946684800000000000, None)) AS date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))]], aggr=[[COUNT(keywords_stream.keyword)]] +03)----LeftSemi Join: keywords_stream.keyword = __correlated_sq_1.keyword +04)------TableScan: keywords_stream projection=[ts, keyword] +05)------SubqueryAlias: __correlated_sq_1 +06)--------TableScan: alert_keywords projection=[keyword] +physical_plan +01)ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))@0 as ts_chunk, COUNT(keywords_stream.keyword)@1 as alert_keyword_count] +02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[COUNT(keywords_stream.keyword)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------MemoryExec: partitions=1, partition_sizes=[1] + +query PI +SELECT + DATE_BIN(INTERVAL '2' MINUTE, ts, '2000-01-01') AS ts_chunk, + COUNT(keyword) AS alert_keyword_count +FROM + keywords_stream +WHERE + keywords_stream.keyword IN (SELECT keyword FROM ALERT_KEYWORDS) +GROUP BY + ts_chunk; +---- +2024-01-01T00:00:00 4 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0cecd7935a74..4b62f2561260 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1382,18 +1382,17 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] -06)----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -09)----------------CoalesceBatchesExec: target_batch_size=2 -10)------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------MemoryExec: partitions=1, partition_sizes=[1] -13)----------------CoalesceBatchesExec: target_batch_size=2 -14)------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -15)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -16)----------------------MemoryExec: partitions=1, partition_sizes=[1] +05)--------AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as alias1], aggr=[] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------CoalesceBatchesExec: target_batch_size=2 +13)----------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +14)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; From 8aad936e3b80833ccb70f2afabeb104b34182190 Mon Sep 17 00:00:00 2001 From: Lorrens Pantelis <100197010+LorrensP-2158466@users.noreply.github.com> Date: Fri, 21 Jun 2024 18:46:22 +0200 Subject: [PATCH 43/54] Compute gcd with u64 instead of i64 because of overflows (#11036) * compute gcd with unsigned ints * add test for the i64::MAX cases * move unsigned_abs below zero test to remove unnecessary casts * add slt test for gcd on max values instead of unit tests --- datafusion/functions/src/math/gcd.rs | 17 +++++++++-------- datafusion/sqllogictest/test_files/scalar.slt | 10 ++++++++++ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/datafusion/functions/src/math/gcd.rs b/datafusion/functions/src/math/gcd.rs index 41c9e4e23314..d0199f7a22c4 100644 --- a/datafusion/functions/src/math/gcd.rs +++ b/datafusion/functions/src/math/gcd.rs @@ -88,16 +88,16 @@ fn gcd(args: &[ArrayRef]) -> Result { /// Computes greatest common divisor using Binary GCD algorithm. pub fn compute_gcd(x: i64, y: i64) -> i64 { - let mut a = x.wrapping_abs(); - let mut b = y.wrapping_abs(); - - if a == 0 { - return b; + if x == 0 { + return y; } - if b == 0 { - return a; + if y == 0 { + return x; } + let mut a = x.unsigned_abs(); + let mut b = y.unsigned_abs(); + let shift = (a | b).trailing_zeros(); a >>= shift; b >>= shift; @@ -112,7 +112,8 @@ pub fn compute_gcd(x: i64, y: i64) -> i64 { b -= a; if b == 0 { - return a << shift; + // because the input values are i64, casting this back to i64 is safe + return (a << shift) as i64; } } } diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index 04b0fa8acab0..551c50e0a17b 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -474,6 +474,16 @@ select gcd(null, null); ---- NULL +# scalar maxes and/or negative 1 +query III rowsort +select + gcd(9223372036854775807, -9223372036854775808), -- i64::MIN, i64::MAX + -- wait till fix, cause it fails gcd(-9223372036854775808, -9223372036854775808), -- -i64::MIN, i64::MIN + gcd(9223372036854775807, -1), -- i64::MAX, -1 + gcd(-9223372036854775808, -1); -- i64::MIN, -1 +---- +1 1 1 + # gcd with columns query III rowsort select gcd(a, b), gcd(c, d), gcd(e, f) from signed_integers; From 30a6ed557d4c695c00de4f081cef36c4122e9f7f Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Fri, 21 Jun 2024 13:13:48 -0400 Subject: [PATCH 44/54] Add distinct_on to dataframe api (#11012) * Add distinct_on to dataframe api #11011 * cargo fmt * Update datafusion/core/src/dataframe/mod.rs as per reviewer feedback Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- datafusion/core/src/dataframe/mod.rs | 117 +++++++++++++++++++++++++++ docs/source/user-guide/dataframe.md | 1 + 2 files changed, 118 insertions(+) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 43ac3992be78..398f59e35d10 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -522,6 +522,38 @@ impl DataFrame { }) } + /// Return a new `DataFrame` with duplicated rows removed as per the specified expression list + /// according to the provided sorting expressions grouped by the `DISTINCT ON` clause + /// expressions. + /// + /// # Example + /// ``` + /// # use datafusion::prelude::*; + /// # use datafusion::error::Result; + /// # #[tokio::main] + /// # async fn main() -> Result<()> { + /// let ctx = SessionContext::new(); + /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await? + /// // Return a single row (a, b) for each distinct value of a + /// .distinct_on(vec![col("a")], vec![col("a"), col("b")], None)?; + /// # Ok(()) + /// # } + /// ``` + pub fn distinct_on( + self, + on_expr: Vec, + select_expr: Vec, + sort_expr: Option>, + ) -> Result { + let plan = LogicalPlanBuilder::from(self.plan) + .distinct_on(on_expr, select_expr, sort_expr)? + .build()?; + Ok(DataFrame { + session_state: self.session_state, + plan, + }) + } + /// Return a new `DataFrame` that has statistics for a DataFrame. /// /// Only summarizes numeric datatypes at the moment and returns nulls for @@ -2359,6 +2391,91 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_distinct_on() -> Result<()> { + let t = test_table().await?; + let plan = t + .distinct_on(vec![col("c1")], vec![col("aggregate_test_100.c1")], None) + .unwrap(); + + let sql_plan = + create_plan("select distinct on (c1) c1 from aggregate_test_100").await?; + + assert_same_plan(&plan.plan.clone(), &sql_plan); + + let df_results = plan.clone().collect().await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!( + ["+----+", + "| c1 |", + "+----+", + "| a |", + "| b |", + "| c |", + "| d |", + "| e |", + "+----+"], + &df_results + ); + + Ok(()) + } + + #[tokio::test] + async fn test_distinct_on_sort_by() -> Result<()> { + let t = test_table().await?; + let plan = t + .select(vec![col("c1")]) + .unwrap() + .distinct_on( + vec![col("c1")], + vec![col("c1")], + Some(vec![col("c1").sort(true, true)]), + ) + .unwrap() + .sort(vec![col("c1").sort(true, true)]) + .unwrap(); + + let df_results = plan.clone().collect().await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!( + ["+----+", + "| c1 |", + "+----+", + "| a |", + "| b |", + "| c |", + "| d |", + "| e |", + "+----+"], + &df_results + ); + + Ok(()) + } + + #[tokio::test] + async fn test_distinct_on_sort_by_unprojected() -> Result<()> { + let t = test_table().await?; + let err = t + .select(vec![col("c1")]) + .unwrap() + .distinct_on( + vec![col("c1")], + vec![col("c1")], + Some(vec![col("c1").sort(true, true)]), + ) + .unwrap() + // try to sort on some value not present in input to distinct + .sort(vec![col("c2").sort(true, true)]) + .unwrap_err(); + assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions c2 must appear in select list"); + + Ok(()) + } + #[tokio::test] async fn join() -> Result<()> { let left = test_table().await?.select_columns(&["c1", "c2"])?; diff --git a/docs/source/user-guide/dataframe.md b/docs/source/user-guide/dataframe.md index 744a719e77be..f011e68fadb2 100644 --- a/docs/source/user-guide/dataframe.md +++ b/docs/source/user-guide/dataframe.md @@ -64,6 +64,7 @@ execution. The plan is evaluated (executed) when an action method is invoked, su | ------------------- | ------------------------------------------------------------------------------------------------------------------------------------------ | | aggregate | Perform an aggregate query with optional grouping expressions. | | distinct | Filter out duplicate rows. | +| distinct_on | Filter out duplicate rows based on provided expressions. | | drop_columns | Create a projection with all but the provided column names. | | except | Calculate the exception of two DataFrames. The two DataFrames must have exactly the same schema | | filter | Filter a DataFrame to only include rows that match the specified filter expression. | From ce4940d0c81e3a1a665bdc5908089c168552fcbd Mon Sep 17 00:00:00 2001 From: Chunchun Ye <14298407+appletreeisyellow@users.noreply.github.com> Date: Fri, 21 Jun 2024 17:30:17 -0500 Subject: [PATCH 45/54] chore: add test to show current behavior of string to timezone vs. timestamp to timezone (#11056) --- .../sqllogictest/test_files/timestamps.slt | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 96d846d449e1..2216dbfa5fd5 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2774,6 +2774,26 @@ SELECT '2000-12-01 04:04:12' AT TIME ZONE 'America/New_York'; ---- 2000-12-01T04:04:12-05:00 +query P +SELECT '2024-03-30 00:00:20' AT TIME ZONE 'Europe/Brussels'; +---- +2024-03-30T00:00:20+01:00 + +query P +SELECT '2024-03-30 00:00:20'::timestamp AT TIME ZONE 'Europe/Brussels'; +---- +2024-03-30T00:00:20+01:00 + +query P +SELECT '2024-03-30 00:00:20Z' AT TIME ZONE 'Europe/Brussels'; +---- +2024-03-30T01:00:20+01:00 + +query P +SELECT '2024-03-30 00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels'; +---- +2024-03-30T00:00:20+01:00 + ## date-time strings that already have a explicit timezone can be used with AT TIME ZONE # same time zone as provided date-time From 4916e891c242220a5eaa02a602e497a03f5cf301 Mon Sep 17 00:00:00 2001 From: Lorrens Pantelis <100197010+LorrensP-2158466@users.noreply.github.com> Date: Sat, 22 Jun 2024 00:30:29 +0200 Subject: [PATCH 46/54] Boolean parquet get datapage stat (#11054) * test and implement boolean data page statistics * left out a collect & forgot to change the Check to Both * Update datafusion/core/src/datasource/physical_plan/parquet/statistics.rs --------- Co-authored-by: Andrew Lamb --- .../physical_plan/parquet/statistics.rs | 26 +++++++++++++++++++ .../core/tests/parquet/arrow_statistics.rs | 2 +- 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index 3be060ce6180..43483a0b6659 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -549,6 +549,18 @@ macro_rules! make_data_page_stats_iterator { }; } +make_data_page_stats_iterator!( + MinBooleanDataPageStatsIterator, + |x: &PageIndex| { x.min }, + Index::BOOLEAN, + bool +); +make_data_page_stats_iterator!( + MaxBooleanDataPageStatsIterator, + |x: &PageIndex| { x.max }, + Index::BOOLEAN, + bool +); make_data_page_stats_iterator!( MinInt32DataPageStatsIterator, |x: &PageIndex| { x.min }, @@ -613,6 +625,15 @@ macro_rules! get_data_page_statistics { ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { paste! { match $data_type { + Some(DataType::Boolean) => Ok(Arc::new( + BooleanArray::from_iter( + [<$stat_type_prefix BooleanDataPageStatsIterator>]::new($iterator) + .flatten() + // BooleanArray::from_iter required a sized iterator, so collect into Vec first + .collect::>() + .into_iter() + ) + )), Some(DataType::UInt8) => Ok(Arc::new( UInt8Array::from_iter( [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) @@ -778,6 +799,11 @@ where { let iter = iterator.flat_map(|(len, index)| match index { Index::NONE => vec![None; len], + Index::BOOLEAN(native_index) => native_index + .indexes + .iter() + .map(|x| x.null_count.map(|x| x as u64)) + .collect::>(), Index::INT32(native_index) => native_index .indexes .iter() diff --git a/datafusion/core/tests/parquet/arrow_statistics.rs b/datafusion/core/tests/parquet/arrow_statistics.rs index ddb39fce4076..cd6985b311c3 100644 --- a/datafusion/core/tests/parquet/arrow_statistics.rs +++ b/datafusion/core/tests/parquet/arrow_statistics.rs @@ -1953,7 +1953,7 @@ async fn test_boolean() { expected_null_counts: UInt64Array::from(vec![1, 0]), expected_row_counts: Some(UInt64Array::from(vec![5, 5])), column_name: "bool", - check: Check::RowGroup, + check: Check::Both, } .run(); } From a4799c093c57aeaacdd4448cb983d79831510483 Mon Sep 17 00:00:00 2001 From: Lordworms <48054792+Lordworms@users.noreply.github.com> Date: Fri, 21 Jun 2024 16:46:52 -0700 Subject: [PATCH 47/54] Using display_name for Expr::Aggregation (#11020) --- datafusion/core/src/physical_planner.rs | 1 + datafusion/sqllogictest/test_files/expr.slt | 37 +++++++++++++++++++-- 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 404bcbb2e7d4..b539544d8372 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1918,6 +1918,7 @@ pub fn create_aggregate_expr_and_maybe_filter( // 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), }; diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index d274d7d4390c..4e8f3b59a650 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2473,7 +2473,7 @@ host2 202 host3 303 # TODO: Issue tracked in https://github.com/apache/datafusion/issues/10364 -query error +query TR select t2.server['c3'] as host, sum(( @@ -2488,6 +2488,10 @@ select ) t2 where t2.server['c3'] IS NOT NULL group by t2.server['c3'] order by host; +---- +host1 101 +host2 202 +host3 303 # can have 2 projections with aggr(short_circuited), with different short-circuited expr query TRR @@ -2559,7 +2563,7 @@ host2 2.2 202 host3 3.3 303 # TODO: Issue tracked in https://github.com/apache/datafusion/issues/10364 -query error +query TRR select t2.server['c3'] as host, sum(( @@ -2579,6 +2583,10 @@ select ) t2 where t2.server['c3'] IS NOT NULL group by t2.server['c3'] order by host; +---- +host1 1.1 101 +host2 2.2 202 +host3 3.3 303 # can have 2 projections with aggr(short_circuited), with the same short-circuited expr (e.g. coalesce) query TRR @@ -2587,3 +2595,28 @@ select t2."struct(t1.time,t1.load1,t1.load2,t1.host)"['c3'] as host, sum(coalesc host1 1.1 101 host2 2.2 202 host3 3.3 303 + +statement ok +set datafusion.sql_parser.dialect = 'Postgres'; + +statement ok +create table t (a float) as values (1), (2), (3); + +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))]] +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))] +03)----MemoryExec: partitions=1, partition_sizes=[1] + + +statement ok +drop table t; + +statement ok +set datafusion.sql_parser.dialect = 'Generic'; From 569be9eb1ba60d3e842f22de5080c732ac2206f4 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 22 Jun 2024 15:19:38 +0800 Subject: [PATCH 48/54] Minor: Convert `Count`'s name to lowercase (#11028) * push down non-unnest only Signed-off-by: jayzhan211 * add doc Signed-off-by: jayzhan211 * to lowercase Signed-off-by: jayzhan211 * fix tpch Signed-off-by: jayzhan211 * Update test * fix test Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/core/src/dataframe/mod.rs | 4 +- datafusion/core/src/execution/context/csv.rs | 4 +- .../aggregate_statistics.rs | 2 +- .../core/tests/custom_sources_cases/mod.rs | 2 +- datafusion/core/tests/dataframe/mod.rs | 10 +-- datafusion/core/tests/path_partition.rs | 4 +- datafusion/core/tests/sql/aggregates.rs | 36 ++++---- datafusion/core/tests/sql/explain_analyze.rs | 8 +- datafusion/functions-aggregate/src/count.rs | 6 +- .../src/analyzer/count_wildcard_rule.rs | 40 ++++----- datafusion/optimizer/src/decorrelate.rs | 2 +- .../src/eliminate_group_by_constant.rs | 20 ++--- .../optimizer/src/optimize_projections/mod.rs | 8 +- .../src/single_distinct_to_groupby.rs | 66 +++++++------- .../optimizer/tests/optimizer_integration.rs | 8 +- .../src/windows/bounded_window_agg_exec.rs | 6 +- datafusion/sql/src/unparser/expr.rs | 6 +- datafusion/sql/tests/sql_integration.rs | 74 +++++++-------- datafusion/sqllogictest/test_files/avro.slt | 6 +- .../sqllogictest/test_files/explain.slt | 2 +- .../sqllogictest/test_files/group_by.slt | 18 ++-- datafusion/sqllogictest/test_files/insert.slt | 28 +++--- .../test_files/insert_to_external.slt | 20 ++--- datafusion/sqllogictest/test_files/joins.slt | 18 ++-- datafusion/sqllogictest/test_files/json.slt | 6 +- datafusion/sqllogictest/test_files/limit.slt | 18 ++-- .../optimizer_group_by_constant.slt | 16 ++-- .../sqllogictest/test_files/predicates.slt | 4 +- datafusion/sqllogictest/test_files/select.slt | 6 +- .../sqllogictest/test_files/subquery.slt | 90 +++++++++---------- .../sqllogictest/test_files/tpch/q1.slt.part | 10 +-- .../sqllogictest/test_files/tpch/q13.slt.part | 18 ++-- .../sqllogictest/test_files/tpch/q16.slt.part | 10 +-- .../sqllogictest/test_files/tpch/q21.slt.part | 10 +-- .../sqllogictest/test_files/tpch/q22.slt.part | 10 +-- .../sqllogictest/test_files/tpch/q4.slt.part | 10 +-- datafusion/sqllogictest/test_files/union.slt | 18 ++-- datafusion/sqllogictest/test_files/window.slt | 82 ++++++++--------- .../substrait/src/logical_plan/consumer.rs | 2 +- .../tests/cases/consumer_integration.rs | 4 +- .../tests/cases/roundtrip_logical_plan.rs | 8 +- 41 files changed, 359 insertions(+), 361 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 398f59e35d10..d55caf1404a5 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2018,7 +2018,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 |", @@ -3171,7 +3171,7 @@ mod tests { let sql = r#" SELECT - COUNT(1) + count(1) FROM test GROUP BY diff --git a/datafusion/core/src/execution/context/csv.rs b/datafusion/core/src/execution/context/csv.rs index aa1a8b512f7e..08e93cb61305 100644 --- a/datafusion/core/src/execution/context/csv.rs +++ b/datafusion/core/src/execution/context/csv.rs @@ -110,12 +110,12 @@ mod tests { ) .await?; let results = - plan_and_collect(&ctx, "SELECT sum(c1), sum(c2), COUNT(*) FROM test").await?; + plan_and_collect(&ctx, "SELECT sum(c1), sum(c2), count(*) FROM test").await?; assert_eq!(results.len(), 1); let expected = [ "+--------------+--------------+----------+", - "| sum(test.c1) | sum(test.c2) | COUNT(*) |", + "| sum(test.c1) | sum(test.c2) | count(*) |", "+--------------+--------------+----------+", "| 10 | 110 | 20 |", "+--------------+--------------+----------+", diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index ca1582bcb34f..7e9aec9e5e4c 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -141,7 +141,7 @@ fn take_optimizable_column_and_table_count( ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name() == "COUNT" && !agg_expr.is_distinct() { + if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { if let Precision::Exact(num_rows) = stats.num_rows { let exprs = agg_expr.expressions(); if exprs.len() == 1 { diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index e8ead01d2ee4..7901998ebd09 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -278,7 +278,7 @@ async fn optimizers_catch_all_statistics() { let expected = RecordBatch::try_new( Arc::new(Schema::new(vec![ - Field::new("COUNT(*)", DataType::Int64, false), + Field::new("count(*)", DataType::Int64, false), Field::new("MIN(test.c1)", DataType::Int32, false), Field::new("MAX(test.c1)", DataType::Int32, false), ])), diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index fa364c5f2a65..c018ebec4b63 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -170,7 +170,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { let ctx = create_join_context()?; let sql_results = ctx - .sql("select COUNT(*) OVER(ORDER BY a DESC RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING) from t1") + .sql("select count(*) OVER(ORDER BY a DESC RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING) from t1") .await? .explain(false, false)? .collect() @@ -211,7 +211,7 @@ async fn test_count_wildcard_on_aggregate() -> Result<()> { let sql_results = ctx .sql("select count(*) from t1") .await? - .select(vec![col("COUNT(*)")])? + .select(vec![col("count(*)")])? .explain(false, false)? .collect() .await?; @@ -604,7 +604,7 @@ async fn test_grouping_sets() -> Result<()> { let expected = vec![ "+-----------+-----+---------------+", - "| a | b | COUNT(test.a) |", + "| a | b | count(test.a) |", "+-----------+-----+---------------+", "| | 100 | 1 |", "| | 10 | 2 |", @@ -645,7 +645,7 @@ async fn test_grouping_sets_count() -> Result<()> { let expected = vec![ "+----+----+-----------------+", - "| c1 | c2 | COUNT(Int32(1)) |", + "| c1 | c2 | count(Int32(1)) |", "+----+----+-----------------+", "| | 5 | 14 |", "| | 4 | 23 |", @@ -1233,7 +1233,7 @@ async fn unnest_aggregate_columns() -> Result<()> { .await?; let expected = [ r#"+-------------+"#, - r#"| COUNT(tags) |"#, + r#"| count(tags) |"#, r#"+-------------+"#, r#"| 9 |"#, r#"+-------------+"#, diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index bfc5b59f0952..7e7544bdb7c0 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -120,7 +120,7 @@ async fn parquet_distinct_partition_col() -> Result<()> { //3. limit is not contained within a single partition //The id column is included to ensure that the parquet file is actually scanned. let results = ctx - .sql("SELECT COUNT(*) as num_rows_per_month, month, MAX(id) from t group by month order by num_rows_per_month desc") + .sql("SELECT count(*) as num_rows_per_month, month, MAX(id) from t group by month order by num_rows_per_month desc") .await? .collect() .await?; @@ -339,7 +339,7 @@ async fn csv_grouping_by_partition() -> Result<()> { let expected = [ "+------------+----------+----------------------+", - "| date | COUNT(*) | COUNT(DISTINCT t.c1) |", + "| date | count(*) | count(DISTINCT t.c1) |", "+------------+----------+----------------------+", "| 2021-10-26 | 100 | 5 |", "| 2021-10-27 | 100 | 5 |", diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 84b791a3de05..c3139f6fcdfb 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -69,12 +69,12 @@ async fn csv_query_array_agg_distinct() -> Result<()> { #[tokio::test] async fn count_partitioned() -> Result<()> { let results = - execute_with_partition("SELECT COUNT(c1), COUNT(c2) FROM test", 4).await?; + execute_with_partition("SELECT count(c1), count(c2) FROM test", 4).await?; assert_eq!(results.len(), 1); let expected = [ "+----------------+----------------+", - "| COUNT(test.c1) | COUNT(test.c2) |", + "| count(test.c1) | count(test.c2) |", "+----------------+----------------+", "| 40 | 40 |", "+----------------+----------------+", @@ -86,11 +86,11 @@ async fn count_partitioned() -> Result<()> { #[tokio::test] async fn count_aggregated() -> Result<()> { let results = - execute_with_partition("SELECT c1, COUNT(c2) FROM test GROUP BY c1", 4).await?; + execute_with_partition("SELECT c1, count(c2) FROM test GROUP BY c1", 4).await?; let expected = [ "+----+----------------+", - "| c1 | COUNT(test.c2) |", + "| c1 | count(test.c2) |", "+----+----------------+", "| 0 | 10 |", "| 1 | 10 |", @@ -105,14 +105,14 @@ async fn count_aggregated() -> Result<()> { #[tokio::test] async fn count_aggregated_cube() -> Result<()> { let results = execute_with_partition( - "SELECT c1, c2, COUNT(c3) FROM test GROUP BY CUBE (c1, c2) ORDER BY c1, c2", + "SELECT c1, c2, count(c3) FROM test GROUP BY CUBE (c1, c2) ORDER BY c1, c2", 4, ) .await?; let expected = vec![ "+----+----+----------------+", - "| c1 | c2 | COUNT(test.c3) |", + "| c1 | c2 | count(test.c3) |", "+----+----+----------------+", "| | | 40 |", "| | 1 | 4 |", @@ -222,15 +222,15 @@ async fn run_count_distinct_integers_aggregated_scenario( " SELECT c_group, - COUNT(c_uint64), - COUNT(DISTINCT c_int8), - COUNT(DISTINCT c_int16), - COUNT(DISTINCT c_int32), - COUNT(DISTINCT c_int64), - COUNT(DISTINCT c_uint8), - COUNT(DISTINCT c_uint16), - COUNT(DISTINCT c_uint32), - COUNT(DISTINCT c_uint64) + count(c_uint64), + count(DISTINCT c_int8), + count(DISTINCT c_int16), + count(DISTINCT c_int32), + count(DISTINCT c_int64), + count(DISTINCT c_uint8), + count(DISTINCT c_uint16), + count(DISTINCT c_uint32), + count(DISTINCT c_uint64) FROM test GROUP BY c_group ", @@ -260,7 +260,7 @@ async fn count_distinct_integers_aggregated_single_partition() -> Result<()> { let results = run_count_distinct_integers_aggregated_scenario(partitions).await?; let expected = ["+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+", - "| c_group | COUNT(test.c_uint64) | COUNT(DISTINCT test.c_int8) | COUNT(DISTINCT test.c_int16) | COUNT(DISTINCT test.c_int32) | COUNT(DISTINCT test.c_int64) | COUNT(DISTINCT test.c_uint8) | COUNT(DISTINCT test.c_uint16) | COUNT(DISTINCT test.c_uint32) | COUNT(DISTINCT test.c_uint64) |", + "| c_group | count(test.c_uint64) | count(DISTINCT test.c_int8) | count(DISTINCT test.c_int16) | count(DISTINCT test.c_int32) | count(DISTINCT test.c_int64) | count(DISTINCT test.c_uint8) | count(DISTINCT test.c_uint16) | count(DISTINCT test.c_uint32) | count(DISTINCT test.c_uint64) |", "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+", "| a | 3 | 2 | 2 | 2 | 2 | 2 | 2 | 2 | 2 |", "| b | 1 | 1 | 1 | 1 | 1 | 1 | 1 | 1 | 1 |", @@ -284,7 +284,7 @@ async fn count_distinct_integers_aggregated_multiple_partitions() -> Result<()> let results = run_count_distinct_integers_aggregated_scenario(partitions).await?; let expected = ["+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+", - "| c_group | COUNT(test.c_uint64) | COUNT(DISTINCT test.c_int8) | COUNT(DISTINCT test.c_int16) | COUNT(DISTINCT test.c_int32) | COUNT(DISTINCT test.c_int64) | COUNT(DISTINCT test.c_uint8) | COUNT(DISTINCT test.c_uint16) | COUNT(DISTINCT test.c_uint32) | COUNT(DISTINCT test.c_uint64) |", + "| c_group | count(test.c_uint64) | count(DISTINCT test.c_int8) | count(DISTINCT test.c_int16) | count(DISTINCT test.c_int32) | count(DISTINCT test.c_int64) | count(DISTINCT test.c_uint8) | count(DISTINCT test.c_uint16) | count(DISTINCT test.c_uint32) | count(DISTINCT test.c_uint64) |", "+---------+----------------------+-----------------------------+------------------------------+------------------------------+------------------------------+------------------------------+-------------------------------+-------------------------------+-------------------------------+", "| a | 5 | 3 | 3 | 3 | 3 | 3 | 3 | 3 | 3 |", "| b | 5 | 4 | 4 | 4 | 4 | 4 | 4 | 4 | 4 |", @@ -301,7 +301,7 @@ async fn test_accumulator_row_accumulator() -> Result<()> { let ctx = SessionContext::new_with_config(config); register_aggregate_csv(&ctx).await?; - let sql = "SELECT c1, c2, MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 + let sql = "SELECT c1, c2, MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, count(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 FROM aggregate_test_100 GROUP BY c1, c2 ORDER BY c1, c2 diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 3e5a0681589c..502590f9e2e2 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -81,7 +81,7 @@ async fn explain_analyze_baseline_metrics() { ); assert_metrics!( &formatted, - "ProjectionExec: expr=[COUNT(*)", + "ProjectionExec: expr=[count(*)", "metrics=[output_rows=1, elapsed_compute=" ); assert_metrics!( @@ -700,7 +700,7 @@ async fn csv_explain_analyze() { // Only test basic plumbing and try to avoid having to change too // many things. explain_analyze_baseline_metrics covers the values // in greater depth - let needle = "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(*)], metrics=[output_rows=5"; + let needle = "AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[count(*)], metrics=[output_rows=5"; assert_contains!(&formatted, needle); let verbose_needle = "Output Rows"; @@ -793,7 +793,7 @@ async fn explain_logical_plan_only() { let expected = vec![ vec![ "logical_plan", - "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]]\ + "Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]]\ \n SubqueryAlias: t\ \n Projection: \ \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))" @@ -812,7 +812,7 @@ async fn explain_physical_plan_only() { let expected = vec![vec![ "physical_plan", - "ProjectionExec: expr=[2 as COUNT(*)]\ + "ProjectionExec: expr=[2 as count(*)]\ \n PlaceholderRowExec\ \n", ]]; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 062e148975bf..84abc0d73098 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -77,7 +77,6 @@ pub fn count_distinct(expr: Expr) -> datafusion_expr::Expr { pub struct Count { signature: Signature, - aliases: Vec, } impl Debug for Count { @@ -98,7 +97,6 @@ impl Default for Count { impl Count { pub fn new() -> Self { Self { - aliases: vec!["count".to_string()], signature: Signature::variadic_any(Volatility::Immutable), } } @@ -110,7 +108,7 @@ impl AggregateUDFImpl for Count { } fn name(&self) -> &str { - "COUNT" + "count" } fn signature(&self) -> &Signature { @@ -249,7 +247,7 @@ impl AggregateUDFImpl for Count { } 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 de2af520053a..34f9802b1fd9 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -59,14 +59,14 @@ fn is_count_star_aggregate(aggregate_function: &AggregateFunction) -> bool { func_def: AggregateFunctionDefinition::UDF(udf), args, .. - } if udf.name() == "COUNT" && args.len() == 1 && is_wildcard(&args[0])) + } if udf.name() == "count" && args.len() == 1 && is_wildcard(&args[0])) } fn is_count_star_window_aggregate(window_function: &WindowFunction) -> bool { let args = &window_function.args; matches!(window_function.fun, WindowFunctionDefinition::AggregateUDF(ref udaf) - if udaf.name() == "COUNT" && args.len() == 1 && is_wildcard(&args[0])) + if udaf.name() == "count" && args.len() == 1 && is_wildcard(&args[0])) } fn analyze_internal(plan: LogicalPlan) -> Result> { @@ -127,9 +127,9 @@ mod tests { .project(vec![count(wildcard())])? .sort(vec![count(wildcard()).sort(true, false)])? .build()?; - let expected = "Sort: COUNT(*) ASC NULLS LAST [COUNT(*):Int64;N]\ - \n Projection: COUNT(*) [COUNT(*):Int64;N]\ - \n Aggregate: groupBy=[[test.b]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] [b:UInt32, COUNT(*):Int64;N]\ + let expected = "Sort: count(*) ASC NULLS LAST [count(*):Int64;N]\ + \n Projection: count(*) [count(*):Int64;N]\ + \n Aggregate: groupBy=[[test.b]], aggr=[[count(Int64(1)) AS count(*)]] [b:UInt32, count(*):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_plan_eq(plan, expected) } @@ -152,9 +152,9 @@ mod tests { .build()?; let expected = "Filter: t1.a IN () [a:UInt32, b:UInt32, c:UInt32]\ - \n Subquery: [COUNT(*):Int64;N]\ - \n Projection: COUNT(*) [COUNT(*):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] [COUNT(*):Int64;N]\ + \n Subquery: [count(*):Int64;N]\ + \n Projection: count(*) [count(*):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] [count(*):Int64;N]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]"; assert_plan_eq(plan, expected) @@ -175,9 +175,9 @@ mod tests { .build()?; let expected = "Filter: EXISTS () [a:UInt32, b:UInt32, c:UInt32]\ - \n Subquery: [COUNT(*):Int64;N]\ - \n Projection: COUNT(*) [COUNT(*):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] [COUNT(*):Int64;N]\ + \n Subquery: [count(*):Int64;N]\ + \n Projection: count(*) [count(*):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] [count(*):Int64;N]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]"; assert_plan_eq(plan, expected) @@ -207,9 +207,9 @@ mod tests { let expected = "Projection: t1.a, t1.b [a:UInt32, b:UInt32]\ \n Filter: () > UInt8(0) [a:UInt32, b:UInt32, c:UInt32]\ - \n Subquery: [COUNT(Int64(1)):Int64;N]\ - \n Projection: COUNT(Int64(1)) [COUNT(Int64(1)):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] [COUNT(Int64(1)):Int64;N]\ + \n Subquery: [count(Int64(1)):Int64;N]\ + \n Projection: count(Int64(1)) [count(Int64(1)):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] [count(Int64(1)):Int64;N]\ \n Filter: outer_ref(t1.a) = t2.a [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]"; @@ -235,8 +235,8 @@ mod tests { .project(vec![count(wildcard())])? .build()?; - let expected = "Projection: COUNT(Int64(1)) AS COUNT(*) [COUNT(*):Int64;N]\ - \n WindowAggr: windowExpr=[[COUNT(Int64(1)) ORDER BY [test.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS COUNT(*) ORDER BY [test.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] [a:UInt32, b:UInt32, c:UInt32, COUNT(*) ORDER BY [test.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING:Int64;N]\ + let expected = "Projection: count(Int64(1)) AS count(*) [count(*):Int64;N]\ + \n WindowAggr: windowExpr=[[count(Int64(1)) ORDER BY [test.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS count(*) ORDER BY [test.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING]] [a:UInt32, b:UInt32, c:UInt32, count(*) ORDER BY [test.a DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING:Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_plan_eq(plan, expected) } @@ -249,8 +249,8 @@ mod tests { .project(vec![count(wildcard())])? .build()?; - let expected = "Projection: COUNT(*) [COUNT(*):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] [COUNT(*):Int64;N]\ + let expected = "Projection: count(*) [count(*):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] [count(*):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_plan_eq(plan, expected) } @@ -272,8 +272,8 @@ mod tests { .project(vec![count(wildcard())])? .build()?; - let expected = "Projection: COUNT(Int64(1)) AS COUNT(*) [COUNT(*):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[MAX(COUNT(Int64(1))) AS MAX(COUNT(*))]] [MAX(COUNT(*)):Int64;N]\ + let expected = "Projection: count(Int64(1)) AS count(*) [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/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index e949e1921b97..7806a622ac0f 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -436,7 +436,7 @@ fn agg_exprs_evaluation_result_on_empty_batch( Transformed::yes(Expr::Literal(ScalarValue::Null)) } AggregateFunctionDefinition::UDF(fun) => { - if fun.name() == "COUNT" { + if fun.name() == "count" { Transformed::yes(Expr::Literal(ScalarValue::Int64(Some( 0, )))) diff --git a/datafusion/optimizer/src/eliminate_group_by_constant.rs b/datafusion/optimizer/src/eliminate_group_by_constant.rs index 7a8dd7aac249..5d9a1193fd78 100644 --- a/datafusion/optimizer/src/eliminate_group_by_constant.rs +++ b/datafusion/optimizer/src/eliminate_group_by_constant.rs @@ -176,8 +176,8 @@ mod tests { .build()?; let expected = "\ - Projection: test.a, UInt32(1), COUNT(test.c)\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.c)]]\ + Projection: test.a, UInt32(1), count(test.c)\ + \n Aggregate: groupBy=[[test.a]], aggr=[[count(test.c)]]\ \n TableScan: test\ "; @@ -196,8 +196,8 @@ mod tests { .build()?; let expected = "\ - Projection: Utf8(\"test\"), UInt32(123), COUNT(test.c)\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(test.c)]]\ + Projection: Utf8(\"test\"), UInt32(123), count(test.c)\ + \n Aggregate: groupBy=[[]], aggr=[[count(test.c)]]\ \n TableScan: test\ "; @@ -216,7 +216,7 @@ mod tests { .build()?; let expected = "\ - Aggregate: groupBy=[[test.a, test.b]], aggr=[[COUNT(test.c)]]\ + Aggregate: groupBy=[[test.a, test.b]], aggr=[[count(test.c)]]\ \n TableScan: test\ "; @@ -257,8 +257,8 @@ mod tests { .build()?; let expected = "\ - Projection: UInt32(123) AS const, test.a, COUNT(test.c)\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.c)]]\ + Projection: UInt32(123) AS const, test.a, count(test.c)\ + \n Aggregate: groupBy=[[test.a]], aggr=[[count(test.c)]]\ \n TableScan: test\ "; @@ -282,8 +282,8 @@ mod tests { .build()?; let expected = "\ - Projection: scalar_fn_mock(UInt32(123)), test.a, COUNT(test.c)\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.c)]]\ + Projection: scalar_fn_mock(UInt32(123)), test.a, count(test.c)\ + \n Aggregate: groupBy=[[test.a]], aggr=[[count(test.c)]]\ \n TableScan: test\ "; @@ -307,7 +307,7 @@ mod tests { .build()?; let expected = "\ - Aggregate: groupBy=[[scalar_fn_mock(UInt32(123)), test.a]], aggr=[[COUNT(test.c)]]\ + Aggregate: groupBy=[[scalar_fn_mock(UInt32(123)), test.a]], aggr=[[count(test.c)]]\ \n TableScan: test\ "; diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 11540d3e162e..5749469f2ddc 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -171,7 +171,7 @@ fn optimize_projections( // still need to create a correct aggregate, which may be optimized // out later. As an example, consider the following query: // - // SELECT COUNT(*) FROM (SELECT COUNT(*) FROM [...]) + // SELECT count(*) FROM (SELECT count(*) FROM [...]) // // which always returns 1. if new_aggr_expr.is_empty() @@ -1049,9 +1049,9 @@ mod tests { .build() .unwrap(); - let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(Int32(1))]]\ + let expected = "Aggregate: groupBy=[[]], aggr=[[count(Int32(1))]]\ \n Projection: \ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int32(1))]]\ + \n Aggregate: groupBy=[[]], aggr=[[count(Int32(1))]]\ \n TableScan: ?table? projection=[]"; assert_optimized_plan_equal(plan, expected) } @@ -1901,7 +1901,7 @@ mod tests { )? .build()?; - let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.b), COUNT(test.b) FILTER (WHERE test.c > Int32(42)) AS count2]]\ + let expected = "Aggregate: groupBy=[[test.a]], aggr=[[count(test.b), count(test.b) FILTER (WHERE test.c > Int32(42)) AS count2]]\ \n TableScan: test projection=[a, b, c]"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index d3d22eb53f39..4c98b9398a5b 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -40,12 +40,12 @@ use hashbrown::HashSet; /// single distinct to group by optimizer rule /// ```text /// Before: -/// SELECT a, COUNT(DINSTINCT b), sum(c) +/// SELECT a, count(DINSTINCT b), sum(c) /// FROM t /// GROUP BY a /// /// After: -/// SELECT a, COUNT(alias1), sum(alias2) +/// SELECT a, count(alias1), sum(alias2) /// FROM ( /// SELECT a, b as alias1, sum(c) as alias2 /// FROM t @@ -175,7 +175,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { // if from parent operators successfully. // Consider plan below. // - // Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ + // Aggregate: groupBy=[[group_alias_0]], aggr=[[count(alias1)]] [group_alias_0:Int32, count(alias1):Int64;N]\ // --Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ // ----TableScan: test [a:UInt32, b:UInt32, c:UInt32] // @@ -183,7 +183,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { // Second aggregate refers to the `group_alias_0` column, Which is a valid field in the first aggregate. // If we were to write plan above as below without alias // - // Aggregate: groupBy=[[test.a + Int32(1)]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ + // Aggregate: groupBy=[[test.a + Int32(1)]], aggr=[[count(alias1)]] [group_alias_0:Int32, count(alias1):Int64;N]\ // --Aggregate: groupBy=[[test.a + Int32(1), test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ // ----TableScan: test [a:UInt32, b:UInt32, c:UInt32] // @@ -404,8 +404,8 @@ mod tests { .build()?; // Should work - let expected = "Projection: COUNT(alias1) AS COUNT(DISTINCT test.b) [COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] [COUNT(alias1):Int64;N]\ + let expected = "Projection: count(alias1) AS count(DISTINCT test.b) [count(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[count(alias1)]] [count(alias1):Int64;N]\ \n Aggregate: groupBy=[[test.b AS alias1]], aggr=[[]] [alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; @@ -427,7 +427,7 @@ mod tests { .build()?; // Should not be optimized - let expected = "Aggregate: groupBy=[[GROUPING SETS ((test.a), (test.b))]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[GROUPING SETS ((test.a), (test.b))]], aggr=[[count(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, count(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -445,7 +445,7 @@ mod tests { .build()?; // Should not be optimized - let expected = "Aggregate: groupBy=[[CUBE (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[CUBE (test.a, test.b)]], aggr=[[count(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, count(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -464,7 +464,7 @@ mod tests { .build()?; // Should not be optimized - let expected = "Aggregate: groupBy=[[ROLLUP (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[ROLLUP (test.a, test.b)]], aggr=[[count(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, count(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -478,8 +478,8 @@ mod tests { .aggregate(Vec::::new(), vec![count_distinct(lit(2) * col("b"))])? .build()?; - let expected = "Projection: COUNT(alias1) AS COUNT(DISTINCT Int32(2) * test.b) [COUNT(DISTINCT Int32(2) * test.b):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] [COUNT(alias1):Int64;N]\ + let expected = "Projection: count(alias1) AS count(DISTINCT Int32(2) * test.b) [count(DISTINCT Int32(2) * test.b):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[count(alias1)]] [count(alias1):Int64;N]\ \n Aggregate: groupBy=[[Int32(2) * test.b AS alias1]], aggr=[[]] [alias1:Int32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; @@ -495,8 +495,8 @@ mod tests { .build()?; // Should work - let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1)]] [a:UInt32, COUNT(alias1):Int64;N]\ + let expected = "Projection: test.a, count(alias1) AS count(DISTINCT test.b) [a:UInt32, count(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[count(alias1)]] [a:UInt32, count(alias1):Int64;N]\ \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; @@ -515,7 +515,7 @@ mod tests { .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(DISTINCT test.b), COUNT(DISTINCT test.c)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[test.a]], aggr=[[count(DISTINCT test.b), count(DISTINCT test.c)]] [a:UInt32, count(DISTINCT test.b):Int64;N, count(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -542,8 +542,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;N, MAX(DISTINCT test.b):UInt32;N]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1), MAX(alias1)]] [a:UInt32, COUNT(alias1):Int64;N, 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;N, MAX(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[count(alias1), MAX(alias1)]] [a:UInt32, count(alias1):Int64;N, 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]"; @@ -562,7 +562,7 @@ mod tests { .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(DISTINCT test.b), COUNT(test.c)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N, COUNT(test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[test.a]], aggr=[[count(DISTINCT test.b), count(test.c)]] [a:UInt32, count(DISTINCT test.b):Int64;N, count(test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -577,8 +577,8 @@ mod tests { .build()?; // Should work - let expected = "Projection: group_alias_0 AS test.a + Int32(1), COUNT(alias1) AS COUNT(DISTINCT test.c) [test.a + Int32(1):Int32, COUNT(DISTINCT test.c):Int64;N]\ - \n Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ + let expected = "Projection: group_alias_0 AS test.a + Int32(1), count(alias1) AS count(DISTINCT test.c) [test.a + Int32(1):Int32, count(DISTINCT test.c):Int64;N]\ + \n Aggregate: groupBy=[[group_alias_0]], aggr=[[count(alias1)]] [group_alias_0:Int32, count(alias1):Int64;N]\ \n Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; @@ -607,8 +607,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;N, 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;N, 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;N, 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;N, 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]"; @@ -626,8 +626,8 @@ 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]\ - \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]\ + 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]\ + \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]\ \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]"; @@ -645,8 +645,8 @@ 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]\ - \n Aggregate: groupBy=[[test.c]], aggr=[[MIN(alias2), COUNT(alias1)]] [c:UInt32, MIN(alias2):UInt32;N, COUNT(alias1):Int64;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]\ + \n Aggregate: groupBy=[[test.c]], aggr=[[MIN(alias2), count(alias1)]] [c:UInt32, MIN(alias2):UInt32;N, count(alias1):Int64;N]\ \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]"; @@ -670,7 +670,7 @@ mod tests { .aggregate(vec![col("c")], vec![expr, count_distinct(col("b"))])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) FILTER (WHERE test.a > Int32(5)), COUNT(DISTINCT test.b)]] [c:UInt32, sum(test.a) FILTER (WHERE test.a > Int32(5)):UInt64;N, COUNT(DISTINCT test.b):Int64;N]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) FILTER (WHERE test.a > Int32(5)), count(DISTINCT test.b)]] [c:UInt32, sum(test.a) FILTER (WHERE test.a > Int32(5)):UInt64;N, count(DISTINCT test.b):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -680,7 +680,7 @@ mod tests { fn distinct_with_filter() -> Result<()> { let table_scan = test_table_scan()?; - // COUNT(DISTINCT a) FILTER (WHERE a > 5) + // count(DISTINCT a) FILTER (WHERE a > 5) let expr = count_udaf() .call(vec![col("a")]) .distinct() @@ -690,7 +690,7 @@ mod tests { .aggregate(vec![col("c")], vec![sum(col("a")), expr])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a), COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5))]] [c:UInt32, sum(test.a):UInt64;N, COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)):Int64;N]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a), count(DISTINCT test.a) FILTER (WHERE test.a > Int32(5))]] [c:UInt32, sum(test.a):UInt64;N, count(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -713,7 +713,7 @@ mod tests { .aggregate(vec![col("c")], vec![expr, count_distinct(col("b"))])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) ORDER BY [test.a], COUNT(DISTINCT test.b)]] [c:UInt32, sum(test.a) ORDER BY [test.a]:UInt64;N, COUNT(DISTINCT test.b):Int64;N]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) ORDER BY [test.a], count(DISTINCT test.b)]] [c:UInt32, sum(test.a) ORDER BY [test.a]:UInt64;N, count(DISTINCT test.b):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -723,7 +723,7 @@ mod tests { fn distinct_with_order_by() -> Result<()> { let table_scan = test_table_scan()?; - // COUNT(DISTINCT a ORDER BY a) + // count(DISTINCT a ORDER BY a) let expr = count_udaf() .call(vec![col("a")]) .distinct() @@ -733,7 +733,7 @@ mod tests { .aggregate(vec![col("c")], vec![sum(col("a")), expr])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a), COUNT(DISTINCT test.a) ORDER BY [test.a ASC NULLS LAST]]] [c:UInt32, sum(test.a):UInt64;N, COUNT(DISTINCT test.a) ORDER BY [test.a ASC NULLS LAST]:Int64;N]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a), count(DISTINCT test.a) ORDER BY [test.a ASC NULLS LAST]]] [c:UInt32, sum(test.a):UInt64;N, count(DISTINCT test.a) ORDER BY [test.a ASC NULLS LAST]:Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) @@ -743,7 +743,7 @@ mod tests { fn aggregate_with_filter_and_order_by() -> Result<()> { let table_scan = test_table_scan()?; - // COUNT(DISTINCT a ORDER BY a) FILTER (WHERE a > 5) + // count(DISTINCT a ORDER BY a) FILTER (WHERE a > 5) let expr = count_udaf() .call(vec![col("a")]) .distinct() @@ -754,7 +754,7 @@ mod tests { .aggregate(vec![col("c")], vec![sum(col("a")), expr])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a), COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a ASC NULLS LAST]]] [c:UInt32, sum(test.a):UInt64;N, COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a ASC NULLS LAST]:Int64;N]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a), count(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a ASC NULLS LAST]]] [c:UInt32, sum(test.a):UInt64;N, count(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a ASC NULLS LAST]:Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index f60bf6609005..70308183306e 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -187,7 +187,7 @@ fn between_date32_plus_interval() -> Result<()> { WHERE col_date32 between '1998-03-18' AND cast('1998-03-18' as date) + INTERVAL '90 days'"; let plan = test_sql(sql)?; let expected = - "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ + "Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]]\ \n Projection: \ \n Filter: test.col_date32 >= Date32(\"1998-03-18\") AND test.col_date32 <= Date32(\"1998-06-16\")\ \n TableScan: test projection=[col_date32]"; @@ -201,7 +201,7 @@ fn between_date64_plus_interval() -> Result<()> { WHERE col_date64 between '1998-03-18T00:00:00' AND cast('1998-03-18' as date) + INTERVAL '90 days'"; let plan = test_sql(sql)?; let expected = - "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ + "Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]]\ \n Projection: \ \n Filter: test.col_date64 >= Date64(\"1998-03-18\") AND test.col_date64 <= Date64(\"1998-06-16\")\ \n TableScan: test projection=[col_date64]"; @@ -257,8 +257,8 @@ fn join_keys_in_subquery_alias_1() { fn push_down_filter_groupby_expr_contains_alias() { let sql = "SELECT * FROM (SELECT (col_int32 + col_uint32) AS c, count(*) FROM test GROUP BY 1) where c > 3"; let plan = test_sql(sql).unwrap(); - let expected = "Projection: test.col_int32 + test.col_uint32 AS c, COUNT(*)\ - \n Aggregate: groupBy=[[test.col_int32 + CAST(test.col_uint32 AS Int32)]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]]\ + let expected = "Projection: test.col_int32 + test.col_uint32 AS c, count(*)\ + \n Aggregate: groupBy=[[test.col_int32 + CAST(test.col_uint32 AS Int32)]], aggr=[[count(Int64(1)) AS count(*)]]\ \n Filter: test.col_int32 + CAST(test.col_uint32 AS Int32) > Int32(3)\ \n TableScan: test projection=[col_int32, col_uint32]"; assert_eq!(expected, format!("{plan:?}")); diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index fc60ab997375..9eb29891703e 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1653,7 +1653,7 @@ mod tests { // // Effectively following query is run on this data // - // SELECT *, COUNT(*) OVER(PARTITION BY duplicated_hash ORDER BY sn RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) + // SELECT *, count(*) OVER(PARTITION BY duplicated_hash ORDER BY sn RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) // FROM test; // // partition `duplicated_hash=2` receives following data from the input @@ -1727,8 +1727,8 @@ mod tests { let plan = projection_exec(window)?; let expected_plan = vec![ - "ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, COUNT([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]@2 as col_2]", - " BoundedWindowAggExec: wdw=[COUNT([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]: Ok(Field { name: \"COUNT([Column { name: \\\"sn\\\", index: 0 }]) PARTITION BY: [[Column { name: \\\"hash\\\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \\\"sn\\\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Linear]", + "ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]@2 as col_2]", + " BoundedWindowAggExec: wdw=[count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]: Ok(Field { name: \"count([Column { name: \\\"sn\\\", index: 0 }]) PARTITION BY: [[Column { name: \\\"hash\\\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \\\"sn\\\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Linear]", " StreamingTableExec: partition_sizes=1, projection=[sn, hash], infinite_source=true, output_ordering=[sn@0 ASC NULLS LAST]", ]; diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 65481aed64f9..7db6956e3244 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1198,7 +1198,7 @@ mod tests { .distinct() .build() .unwrap(), - "COUNT(DISTINCT *)", + "count(DISTINCT *)", ), ( count_udaf() @@ -1206,7 +1206,7 @@ mod tests { .filter(lit(true)) .build() .unwrap(), - "COUNT(*) FILTER (WHERE true)", + "count(*) FILTER (WHERE true)", ), ( Expr::WindowFunction(WindowFunction { @@ -1242,7 +1242,7 @@ mod tests { ), null_treatment: None, }), - r#"COUNT(*) OVER (ORDER BY a DESC NULLS FIRST RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING)"#, + r#"count(*) OVER (ORDER BY a DESC NULLS FIRST RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING)"#, ), (col("a").is_not_null(), r#"a IS NOT NULL"#), (col("a").is_null(), r#"a IS NULL"#), diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 8eb2a2b609e7..3905d7d18424 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -996,12 +996,12 @@ fn select_aggregate_with_having_with_aggregate_not_in_select() { #[test] fn select_aggregate_with_having_referencing_column_not_in_select() { - let sql = "SELECT COUNT(*) + let sql = "SELECT count(*) FROM person 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: COUNT(*)", + "Error during planning: HAVING clause references non-aggregate values: Expression person.first_name could not be resolved from available columns: count(*)", err.strip_backtrace() ); } @@ -1200,10 +1200,10 @@ fn select_aggregate_with_group_by_with_having_using_count_star_not_in_select() { let sql = "SELECT first_name, MAX(age) FROM person GROUP BY first_name - HAVING MAX(age) > 100 AND COUNT(*) < 50"; + 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(*)]]\ + \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); } @@ -1464,15 +1464,15 @@ fn select_simple_aggregate_with_groupby_and_column_is_in_aggregate_and_groupby() #[test] fn select_simple_aggregate_with_groupby_can_use_positions() { quick_test( - "SELECT state, age AS b, COUNT(1) FROM person GROUP BY 1, 2", - "Projection: person.state, person.age AS b, COUNT(Int64(1))\ - \n Aggregate: groupBy=[[person.state, person.age]], aggr=[[COUNT(Int64(1))]]\ + "SELECT state, age AS b, count(1) FROM person GROUP BY 1, 2", + "Projection: person.state, person.age AS b, count(Int64(1))\ + \n Aggregate: groupBy=[[person.state, person.age]], aggr=[[count(Int64(1))]]\ \n TableScan: person", ); quick_test( - "SELECT state, age AS b, COUNT(1) FROM person GROUP BY 2, 1", - "Projection: person.state, person.age AS b, COUNT(Int64(1))\ - \n Aggregate: groupBy=[[person.age, person.state]], aggr=[[COUNT(Int64(1))]]\ + "SELECT state, age AS b, count(1) FROM person GROUP BY 2, 1", + "Projection: person.state, person.age AS b, count(Int64(1))\ + \n Aggregate: groupBy=[[person.age, person.state]], aggr=[[count(Int64(1))]]\ \n TableScan: person", ); } @@ -1633,18 +1633,18 @@ fn test_wildcard() { #[test] fn select_count_one() { - let sql = "SELECT COUNT(1) FROM person"; - let expected = "Projection: COUNT(Int64(1))\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ + let sql = "SELECT count(1) FROM person"; + let expected = "Projection: count(Int64(1))\ + \n Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]]\ \n TableScan: person"; quick_test(sql, expected); } #[test] fn select_count_column() { - let sql = "SELECT COUNT(id) FROM person"; - let expected = "Projection: COUNT(person.id)\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(person.id)]]\ + let sql = "SELECT count(id) FROM person"; + let expected = "Projection: count(person.id)\ + \n Aggregate: groupBy=[[]], aggr=[[count(person.id)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -1814,9 +1814,9 @@ fn select_group_by_columns_not_in_select() { #[test] fn select_group_by_count_star() { - let sql = "SELECT state, COUNT(*) FROM person GROUP BY state"; - let expected = "Projection: person.state, COUNT(*)\ - \n Aggregate: groupBy=[[person.state]], aggr=[[COUNT(*)]]\ + let sql = "SELECT state, count(*) FROM person GROUP BY state"; + let expected = "Projection: person.state, count(*)\ + \n Aggregate: groupBy=[[person.state]], aggr=[[count(*)]]\ \n TableScan: person"; quick_test(sql, expected); @@ -1824,10 +1824,10 @@ fn select_group_by_count_star() { #[test] fn select_group_by_needs_projection() { - let sql = "SELECT COUNT(state), state FROM person GROUP BY state"; + let sql = "SELECT count(state), state FROM person GROUP BY state"; let expected = "\ - Projection: COUNT(person.state), person.state\ - \n Aggregate: groupBy=[[person.state]], aggr=[[COUNT(person.state)]]\ + Projection: count(person.state), person.state\ + \n Aggregate: groupBy=[[person.state]], aggr=[[count(person.state)]]\ \n TableScan: person"; quick_test(sql, expected); @@ -3000,8 +3000,8 @@ fn scalar_subquery_reference_outer_field() { let expected = "Projection: j1.j1_string, j2.j2_string\ \n Filter: j1.j1_id = j2.j2_id - Int64(1) AND j2.j2_id < ()\ \n Subquery:\ - \n Projection: COUNT(*)\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(*)]]\ + \n Projection: count(*)\ + \n Aggregate: groupBy=[[]], aggr=[[count(*)]]\ \n Filter: outer_ref(j2.j2_id) = j1.j1_id AND j1.j1_id = j3.j3_id\ \n CrossJoin:\ \n TableScan: j1\ @@ -3098,19 +3098,19 @@ fn cte_unbalanced_number_of_columns() { #[test] fn aggregate_with_rollup() { let sql = - "SELECT id, state, age, COUNT(*) FROM person GROUP BY id, ROLLUP (state, age)"; - let expected = "Projection: person.id, person.state, person.age, COUNT(*)\ - \n Aggregate: groupBy=[[GROUPING SETS ((person.id), (person.id, person.state), (person.id, person.state, person.age))]], aggr=[[COUNT(*)]]\ + "SELECT id, state, age, count(*) FROM person GROUP BY id, ROLLUP (state, age)"; + let expected = "Projection: person.id, person.state, person.age, count(*)\ + \n Aggregate: groupBy=[[GROUPING SETS ((person.id), (person.id, person.state), (person.id, person.state, person.age))]], aggr=[[count(*)]]\ \n TableScan: person"; quick_test(sql, expected); } #[test] fn aggregate_with_rollup_with_grouping() { - let sql = "SELECT id, state, age, grouping(state), grouping(age), grouping(state) + grouping(age), COUNT(*) \ + let sql = "SELECT id, state, age, grouping(state), grouping(age), grouping(state) + grouping(age), count(*) \ FROM person GROUP BY id, ROLLUP (state, age)"; - let expected = "Projection: person.id, person.state, person.age, GROUPING(person.state), GROUPING(person.age), GROUPING(person.state) + GROUPING(person.age), COUNT(*)\ - \n Aggregate: groupBy=[[GROUPING SETS ((person.id), (person.id, person.state), (person.id, person.state, person.age))]], aggr=[[GROUPING(person.state), GROUPING(person.age), COUNT(*)]]\ + let expected = "Projection: person.id, person.state, person.age, GROUPING(person.state), GROUPING(person.age), GROUPING(person.state) + GROUPING(person.age), count(*)\ + \n Aggregate: groupBy=[[GROUPING SETS ((person.id), (person.id, person.state), (person.id, person.state, person.age))]], aggr=[[GROUPING(person.state), GROUPING(person.age), count(*)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -3140,9 +3140,9 @@ fn rank_partition_grouping() { #[test] fn aggregate_with_cube() { let sql = - "SELECT id, state, age, COUNT(*) FROM person GROUP BY id, CUBE (state, age)"; - let expected = "Projection: person.id, person.state, person.age, COUNT(*)\ - \n Aggregate: groupBy=[[GROUPING SETS ((person.id), (person.id, person.state), (person.id, person.age), (person.id, person.state, person.age))]], aggr=[[COUNT(*)]]\ + "SELECT id, state, age, count(*) FROM person GROUP BY id, CUBE (state, age)"; + let expected = "Projection: person.id, person.state, person.age, count(*)\ + \n Aggregate: groupBy=[[GROUPING SETS ((person.id), (person.id, person.state), (person.id, person.age), (person.id, person.state, person.age))]], aggr=[[count(*)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -3157,9 +3157,9 @@ fn round_decimal() { #[test] fn aggregate_with_grouping_sets() { - let sql = "SELECT id, state, age, COUNT(*) FROM person GROUP BY id, GROUPING SETS ((state), (state, age), (id, state))"; - let expected = "Projection: person.id, person.state, person.age, COUNT(*)\ - \n Aggregate: groupBy=[[GROUPING SETS ((person.id, person.state), (person.id, person.state, person.age), (person.id, person.id, person.state))]], aggr=[[COUNT(*)]]\ + let sql = "SELECT id, state, age, count(*) FROM person GROUP BY id, GROUPING SETS ((state), (state, age), (id, state))"; + let expected = "Projection: person.id, person.state, person.age, count(*)\ + \n Aggregate: groupBy=[[GROUPING SETS ((person.id, person.state), (person.id, person.state, person.age), (person.id, person.id, person.state))]], aggr=[[count(*)]]\ \n TableScan: person"; quick_test(sql, expected); } diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index fced1924ced9..f8ef81a8ba2b 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -243,11 +243,11 @@ query TT EXPLAIN SELECT count(*) from alltypes_plain ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 02)--TableScan: alltypes_plain projection=[] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]} diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 3c5f8c7f7ad6..96e73a591678 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -93,7 +93,7 @@ query TT EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3) ---- physical_plan -01)ProjectionExec: expr=[2 as COUNT(*)] +01)ProjectionExec: expr=[2 as count(*)] 02)--PlaceholderRowExec statement ok diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index fff3977fe1e6..b08df0e4b35f 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4394,18 +4394,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)]] +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 @@ -5109,15 +5109,15 @@ GROUP BY ts_chunk; ---- logical_plan -01)Projection: date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01")) AS ts_chunk, COUNT(keywords_stream.keyword) AS alert_keyword_count -02)--Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"), keywords_stream.ts, TimestampNanosecond(946684800000000000, None)) AS date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))]], aggr=[[COUNT(keywords_stream.keyword)]] +01)Projection: date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01")) AS ts_chunk, count(keywords_stream.keyword) AS alert_keyword_count +02)--Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"), keywords_stream.ts, TimestampNanosecond(946684800000000000, None)) AS date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))]], aggr=[[count(keywords_stream.keyword)]] 03)----LeftSemi Join: keywords_stream.keyword = __correlated_sq_1.keyword 04)------TableScan: keywords_stream projection=[ts, keyword] 05)------SubqueryAlias: __correlated_sq_1 06)--------TableScan: alert_keywords projection=[keyword] physical_plan -01)ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))@0 as ts_chunk, COUNT(keywords_stream.keyword)@1 as alert_keyword_count] -02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[COUNT(keywords_stream.keyword)] +01)ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))@0 as ts_chunk, count(keywords_stream.keyword)@1 as alert_keyword_count] +02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] 05)--------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 1fa319111c45..9115cb532540 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -58,17 +58,17 @@ ORDER by c1 ---- logical_plan 01)Dml: op=[Insert Into] table=[table_without_values] -02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 03)----Sort: aggregate_test_100.c1 ASC NULLS LAST -04)------Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 -05)--------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +04)------Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 +05)--------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 06)----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan 01)DataSinkExec: sink=MemoryTable (partitions=1) -02)--ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] +02)--ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] -04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] -05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] +04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] +05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 @@ -121,14 +121,14 @@ FROM aggregate_test_100 ---- logical_plan 01)Dml: op=[Insert Into] table=[table_without_values] -02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 -03)----WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +03)----WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 04)------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan 01)DataSinkExec: sink=MemoryTable (partitions=1) 02)--CoalescePartitionsExec -03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] -04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] +04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 @@ -171,15 +171,15 @@ logical_plan 01)Dml: op=[Insert Into] table=[table_without_values] 02)--Projection: a1 AS a1, a2 AS a2 03)----Sort: aggregate_test_100.c1 ASC NULLS LAST -04)------Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a2, aggregate_test_100.c1 -05)--------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +04)------Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a2, aggregate_test_100.c1 +05)--------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 06)----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan 01)DataSinkExec: sink=MemoryTable (partitions=8) 02)--ProjectionExec: expr=[a1@0 as a1, a2@1 as a2] 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] -04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] -05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] +04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] +05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 9f930defbbf9..8f6bafd92e41 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -347,17 +347,17 @@ ORDER by c1 ---- logical_plan 01)Dml: op=[Insert Into] table=[table_without_values] -02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 03)----Sort: aggregate_test_100.c1 ASC NULLS LAST -04)------Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 -05)--------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +04)------Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 +05)--------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 06)----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] +02)--ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] -04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] -05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] +04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] +05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] 06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 @@ -411,14 +411,14 @@ FROM aggregate_test_100 ---- logical_plan 01)Dml: op=[Insert Into] table=[table_without_values] -02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 -03)----WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +02)--Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +03)----WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 04)------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) 02)--CoalescePartitionsExec -03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] -04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] +04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 4b62f2561260..c2949b98aa77 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1343,15 +1343,15 @@ from (select * from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id) group by t1_id ---- logical_plan -01)Projection: COUNT(*) -02)--Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Projection: count(*) +02)--Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[count(Int64(1)) AS count(*)]] 03)----Projection: join_t1.t1_id 04)------Inner Join: join_t1.t1_id = join_t2.t2_id 05)--------TableScan: join_t1 projection=[t1_id] 06)--------TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] -02)--AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] +01)ProjectionExec: expr=[count(*)@1 as count(*)] +02)--AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[count(*)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 05)--------CoalesceBatchesExec: target_batch_size=2 @@ -1370,18 +1370,18 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -01)Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) -02)--Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] +01)Projection: count(alias1) AS count(DISTINCT join_t1.t1_id) +02)--Aggregate: groupBy=[[]], aggr=[[count(alias1)]] 03)----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] 04)------Projection: join_t1.t1_id 05)--------Inner Join: join_t1.t1_id = join_t2.t2_id 06)----------TableScan: join_t1 projection=[t1_id] 07)----------TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] -02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] +01)ProjectionExec: expr=[count(alias1)@0 as count(DISTINCT join_t1.t1_id)] +02)--AggregateExec: mode=Final, gby=[], aggr=[count(alias1)] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] +04)------AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)] 05)--------AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 5d3c23d5130b..0b9508310b00 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -49,12 +49,12 @@ query TT EXPLAIN SELECT count(*) from json_test ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 02)--TableScan: json_test projection=[] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]} diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 2c65b1da4474..094017c383a6 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -307,11 +307,11 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 02)--Limit: skip=11, fetch=3 03)----TableScan: t1 projection=[], fetch=14 physical_plan -01)ProjectionExec: expr=[0 as COUNT(*)] +01)ProjectionExec: expr=[0 as count(*)] 02)--PlaceholderRowExec query I @@ -325,11 +325,11 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 02)--Limit: skip=8, fetch=3 03)----TableScan: t1 projection=[], fetch=11 physical_plan -01)ProjectionExec: expr=[2 as COUNT(*)] +01)ProjectionExec: expr=[2 as count(*)] 02)--PlaceholderRowExec query I @@ -343,11 +343,11 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 OFFSET 8); ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 02)--Limit: skip=8, fetch=None 03)----TableScan: t1 projection=[] physical_plan -01)ProjectionExec: expr=[2 as COUNT(*)] +01)ProjectionExec: expr=[2 as count(*)] 02)--PlaceholderRowExec query I @@ -360,15 +360,15 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); ---- logical_plan -01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 02)--Projection: 03)----Limit: skip=6, fetch=3 04)------Filter: t1.a > Int32(3) 05)--------TableScan: t1 projection=[a] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------ProjectionExec: expr=[] 06)----------GlobalLimitExec: skip=6, fetch=3 diff --git a/datafusion/sqllogictest/test_files/optimizer_group_by_constant.slt b/datafusion/sqllogictest/test_files/optimizer_group_by_constant.slt index f578b08482ac..ef77bdca1b3e 100644 --- a/datafusion/sqllogictest/test_files/optimizer_group_by_constant.slt +++ b/datafusion/sqllogictest/test_files/optimizer_group_by_constant.slt @@ -48,8 +48,8 @@ FROM test_table t GROUP BY 1, 2, 3, 4 ---- logical_plan -01)Projection: t.c1, Int64(99999), t.c5 + t.c8, Utf8("test"), COUNT(Int64(1)) -02)--Aggregate: groupBy=[[t.c1, t.c5 + t.c8]], aggr=[[COUNT(Int64(1))]] +01)Projection: t.c1, Int64(99999), t.c5 + t.c8, Utf8("test"), count(Int64(1)) +02)--Aggregate: groupBy=[[t.c1, t.c5 + t.c8]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: t 04)------TableScan: test_table projection=[c1, c5, c8] @@ -60,8 +60,8 @@ FROM test_table t group by 1, 2, 3 ---- logical_plan -01)Projection: Int64(123), Int64(456), Int64(789), COUNT(Int64(1)), AVG(t.c12) -02)--Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)), AVG(t.c12)]] +01)Projection: Int64(123), Int64(456), Int64(789), count(Int64(1)), AVG(t.c12) +02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1)), AVG(t.c12)]] 03)----SubqueryAlias: t 04)------TableScan: test_table projection=[c12] @@ -72,8 +72,8 @@ FROM test_table t GROUP BY 1, 2 ---- logical_plan -01)Projection: Date32("2023-05-04") AS dt, Boolean(true) AS today_filter, COUNT(Int64(1)) -02)--Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] +01)Projection: Date32("2023-05-04") AS dt, Boolean(true) AS today_filter, count(Int64(1)) +02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: t 04)------TableScan: test_table projection=[] @@ -90,8 +90,8 @@ FROM test_table t GROUP BY 1 ---- logical_plan -01)Projection: Boolean(true) AS NOT date_part(Utf8("MONTH"),now()) BETWEEN Int64(50) AND Int64(60), COUNT(Int64(1)) -02)--Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] +01)Projection: Boolean(true) AS NOT date_part(Utf8("MONTH"),now()) BETWEEN Int64(50) AND Int64(60), count(Int64(1)) +02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 03)----SubqueryAlias: t 04)------TableScan: test_table projection=[] diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index ac0dc3018879..4e8f0eb6caaa 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -748,7 +748,7 @@ OR GROUP BY p_partkey; ---- logical_plan -01)Aggregate: groupBy=[[part.p_partkey]], aggr=[[sum(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)]] +01)Aggregate: groupBy=[[part.p_partkey]], aggr=[[sum(lineitem.l_extendedprice), AVG(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)]] 02)--Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, partsupp.ps_suppkey 03)----Inner Join: part.p_partkey = partsupp.ps_partkey 04)------Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey @@ -759,7 +759,7 @@ logical_plan 09)--------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8("Brand#12") OR part.p_brand = Utf8("Brand#23")] 10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan -01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[sum(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] +01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[sum(lineitem.l_extendedprice), AVG(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] 04)------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index c8ef2b7f5e0b..f9baf8db69d5 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1383,16 +1383,16 @@ ORDER BY c1, c2) GROUP BY c2; ---- logical_plan -01)Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[count(Int64(1)) AS count(*)]] 02)--Projection: aggregate_test_100.c2 03)----Sort: aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST 04)------Projection: aggregate_test_100.c2, aggregate_test_100.c1 05)--------TableScan: aggregate_test_100 projection=[c1, c2] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[COUNT(*)] +01)AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[count(*)] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 -04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[COUNT(*)] +04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(*)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index dbdb7fc76b8b..f56729f85e1c 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -530,9 +530,9 @@ logical_plan 01)Projection: t1.t1_id, t1.t1_name 02)--Filter: EXISTS () 03)----Subquery: -04)------Projection: COUNT(*) +04)------Projection: count(*) 05)--------Filter: sum(outer_ref(t1.t1_int) + t2.t2_id) > Int64(0) -06)----------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*), sum(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] +06)----------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*), sum(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] 07)------------Filter: outer_ref(t1.t1_name) = t2.t2_name 08)--------------TableScan: t2 09)----TableScan: t1 projection=[t1_id, t1_name, t1_int] @@ -717,9 +717,9 @@ query TT explain select (select count(*) from t1) as b ---- logical_plan -01)Projection: __scalar_sq_1.COUNT(*) AS b +01)Projection: __scalar_sq_1.count(*) AS b 02)--SubqueryAlias: __scalar_sq_1 -03)----Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 04)------TableScan: t1 projection=[] #simple_uncorrelated_scalar_subquery2 @@ -727,13 +727,13 @@ query TT explain select (select count(*) from t1) as b, (select count(1) from t2) ---- logical_plan -01)Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT(Int64(1)) +01)Projection: __scalar_sq_1.count(*) AS b, __scalar_sq_2.count(Int64(1)) AS count(Int64(1)) 02)--Left Join: 03)----SubqueryAlias: __scalar_sq_1 -04)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 05)--------TableScan: t1 projection=[] 06)----SubqueryAlias: __scalar_sq_2 -07)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] +07)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 08)--------TableScan: t2 projection=[] statement ok @@ -743,20 +743,20 @@ query TT explain select (select count(*) from t1) as b, (select count(1) from t2) ---- logical_plan -01)Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT(Int64(1)) +01)Projection: __scalar_sq_1.count(*) AS b, __scalar_sq_2.count(Int64(1)) AS count(Int64(1)) 02)--Left Join: 03)----SubqueryAlias: __scalar_sq_1 -04)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 05)--------TableScan: t1 projection=[] 06)----SubqueryAlias: __scalar_sq_2 -07)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] +07)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] 08)--------TableScan: t2 projection=[] physical_plan -01)ProjectionExec: expr=[COUNT(*)@0 as b, COUNT(Int64(1))@1 as COUNT(Int64(1))] +01)ProjectionExec: expr=[count(*)@0 as b, count(Int64(1))@1 as count(Int64(1))] 02)--NestedLoopJoinExec: join_type=Left -03)----ProjectionExec: expr=[4 as COUNT(*)] +03)----ProjectionExec: expr=[4 as count(*)] 04)------PlaceholderRowExec -05)----ProjectionExec: expr=[4 as COUNT(Int64(1))] +05)----ProjectionExec: expr=[4 as count(Int64(1))] 06)------PlaceholderRowExec statement ok @@ -772,12 +772,12 @@ query TT explain SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) from t1 ---- logical_plan -01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END AS COUNT(*) +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END AS count(*) 02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int 03)----TableScan: t1 projection=[t1_id, t1_int] 04)----SubqueryAlias: __scalar_sq_1 -05)------Projection: COUNT(*), t2.t2_int, Boolean(true) AS __always_true -06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)------Projection: count(*), t2.t2_int, Boolean(true) AS __always_true +06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 07)----------TableScan: t2 projection=[t2_int] query II rowsort @@ -794,12 +794,12 @@ query TT explain SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) as cnt from t1 ---- logical_plan -01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END AS cnt +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END AS cnt 02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int 03)----TableScan: t1 projection=[t1_id, t1_int] 04)----SubqueryAlias: __scalar_sq_1 -05)------Projection: COUNT(*), t2.t2_int, Boolean(true) AS __always_true -06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)------Projection: count(*), t2.t2_int, Boolean(true) AS __always_true +06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 07)----------TableScan: t2 projection=[t2_int] query II rowsort @@ -819,8 +819,8 @@ logical_plan 02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int 03)----TableScan: t1 projection=[t1_id, t1_int] 04)----SubqueryAlias: __scalar_sq_1 -05)------Projection: COUNT(*) AS _cnt, t2.t2_int, Boolean(true) AS __always_true -06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)------Projection: count(*) AS _cnt, t2.t2_int, Boolean(true) AS __always_true +06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 07)----------TableScan: t2 projection=[t2_int] query II rowsort @@ -840,8 +840,8 @@ logical_plan 02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int 03)----TableScan: t1 projection=[t1_id, t1_int] 04)----SubqueryAlias: __scalar_sq_1 -05)------Projection: COUNT(*) + Int64(2) AS _cnt, t2.t2_int, Boolean(true) AS __always_true -06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)------Projection: count(*) + Int64(2) AS _cnt, t2.t2_int, Boolean(true) AS __always_true +06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 07)----------TableScan: t2 projection=[t2_int] query II rowsort @@ -858,13 +858,13 @@ explain select t1.t1_int from t1 where (select count(*) from t2 where t1.t1_id = ---- logical_plan 01)Projection: t1.t1_int -02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END < CAST(t1.t1_int AS Int64) -03)----Projection: t1.t1_int, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END < CAST(t1.t1_int AS Int64) +03)----Projection: t1.t1_int, __scalar_sq_1.count(*), __scalar_sq_1.__always_true 04)------Left Join: t1.t1_id = __scalar_sq_1.t2_id 05)--------TableScan: t1 projection=[t1_id, t1_int] 06)--------SubqueryAlias: __scalar_sq_1 -07)----------Projection: COUNT(*), t2.t2_id, Boolean(true) AS __always_true -08)------------Aggregate: groupBy=[[t2.t2_id]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------Projection: count(*), t2.t2_id, Boolean(true) AS __always_true +08)------------Aggregate: groupBy=[[t2.t2_id]], aggr=[[count(Int64(1)) AS count(*)]] 09)--------------TableScan: t2 projection=[t2_id] query I rowsort @@ -884,9 +884,9 @@ logical_plan 02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int 03)----TableScan: t1 projection=[t1_id, t1_int] 04)----SubqueryAlias: __scalar_sq_1 -05)------Projection: COUNT(*) + Int64(2) AS cnt_plus_2, t2.t2_int -06)--------Filter: COUNT(*) > Int64(1) -07)----------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)------Projection: count(*) + Int64(2) AS cnt_plus_2, t2.t2_int +06)--------Filter: count(*) > Int64(1) +07)----------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 08)------------TableScan: t2 projection=[t2_int] query II rowsort @@ -903,12 +903,12 @@ query TT explain SELECT t1_id, (SELECT count(*) + 2 as cnt_plus_2 FROM t2 WHERE t2.t2_int = t1.t1_int having count(*) = 0) from t1 ---- logical_plan -01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) AS cnt_plus_2 WHEN __scalar_sq_1.COUNT(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_2 END AS cnt_plus_2 +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) AS cnt_plus_2 WHEN __scalar_sq_1.count(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_2 END AS cnt_plus_2 02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int 03)----TableScan: t1 projection=[t1_id, t1_int] 04)----SubqueryAlias: __scalar_sq_1 -05)------Projection: COUNT(*) + Int64(2) AS cnt_plus_2, t2.t2_int, COUNT(*), Boolean(true) AS __always_true -06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)------Projection: count(*) + Int64(2) AS cnt_plus_2, t2.t2_int, count(*), Boolean(true) AS __always_true +06)--------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 07)----------TableScan: t2 projection=[t2_int] query II rowsort @@ -925,14 +925,14 @@ explain select t1.t1_int from t1 group by t1.t1_int having (select count(*) from ---- logical_plan 01)Projection: t1.t1_int -02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END = Int64(0) -03)----Projection: t1.t1_int, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END = Int64(0) +03)----Projection: t1.t1_int, __scalar_sq_1.count(*), __scalar_sq_1.__always_true 04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int 05)--------Aggregate: groupBy=[[t1.t1_int]], aggr=[[]] 06)----------TableScan: t1 projection=[t1_int] 07)--------SubqueryAlias: __scalar_sq_1 -08)----------Projection: COUNT(*), t2.t2_int, Boolean(true) AS __always_true -09)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +08)----------Projection: count(*), t2.t2_int, Boolean(true) AS __always_true +09)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 10)--------------TableScan: t2 projection=[t2_int] query I rowsort @@ -952,8 +952,8 @@ logical_plan 04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int 05)--------TableScan: t1 projection=[t1_int] 06)--------SubqueryAlias: __scalar_sq_1 -07)----------Projection: COUNT(*) AS cnt, t2.t2_int, Boolean(true) AS __always_true -08)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------Projection: count(*) AS cnt, t2.t2_int, Boolean(true) AS __always_true +08)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 09)--------------TableScan: t2 projection=[t2_int] @@ -977,13 +977,13 @@ select t1.t1_int from t1 where ( ---- logical_plan 01)Projection: t1.t1_int -02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) WHEN __scalar_sq_1.COUNT(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_two END = Int64(2) -03)----Projection: t1.t1_int, __scalar_sq_1.cnt_plus_two, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) WHEN __scalar_sq_1.count(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_two END = Int64(2) +03)----Projection: t1.t1_int, __scalar_sq_1.cnt_plus_two, __scalar_sq_1.count(*), __scalar_sq_1.__always_true 04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int 05)--------TableScan: t1 projection=[t1_int] 06)--------SubqueryAlias: __scalar_sq_1 -07)----------Projection: COUNT(*) + Int64(1) + Int64(1) AS cnt_plus_two, t2.t2_int, COUNT(*), Boolean(true) AS __always_true -08)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------Projection: count(*) + Int64(1) + Int64(1) AS cnt_plus_two, t2.t2_int, count(*), Boolean(true) AS __always_true +08)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 09)--------------TableScan: t2 projection=[t2_int] query I rowsort @@ -1011,8 +1011,8 @@ logical_plan 04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int 05)--------TableScan: t1 projection=[t1_int] 06)--------SubqueryAlias: __scalar_sq_1 -07)----------Projection: CASE WHEN COUNT(*) = Int64(1) THEN Int64(NULL) ELSE COUNT(*) END AS cnt, t2.t2_int, Boolean(true) AS __always_true -08)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------Projection: CASE WHEN count(*) = Int64(1) THEN Int64(NULL) ELSE count(*) END AS cnt, t2.t2_int, Boolean(true) AS __always_true +08)------------Aggregate: groupBy=[[t2.t2_int]], aggr=[[count(Int64(1)) AS count(*)]] 09)--------------TableScan: t2 projection=[t2_int] diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 5a21bdf276e3..f005081b155f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -41,19 +41,19 @@ explain select ---- logical_plan 01)Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST -02)--Projection: lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity) AS sum_qty, sum(lineitem.l_extendedprice) AS sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(*) AS count_order -03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(__common_expr_1) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(Int64(1)) AS COUNT(*)]] +02)--Projection: lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity) AS sum_qty, sum(lineitem.l_extendedprice) AS sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, count(*) AS count_order +03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(__common_expr_1) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(__common_expr_1 * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), count(Int64(1)) AS count(*)]] 04)------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS __common_expr_1, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus 05)--------Filter: lineitem.l_shipdate <= Date32("1998-09-02") 06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-02")] physical_plan 01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] 02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(*)@9 as count_order] -04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] +03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, count(*)@9 as count_order] +04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] +07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), count(*)] 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: l_shipdate@6 <= 1998-09-02 diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index f19db720fb2c..f25f23de8817 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -42,11 +42,11 @@ limit 10; logical_plan 01)Limit: skip=0, fetch=10 02)--Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST, fetch=10 -03)----Projection: c_orders.c_count, COUNT(*) AS custdist -04)------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----Projection: c_orders.c_count, count(*) AS custdist +04)------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[count(Int64(1)) AS count(*)]] 05)--------SubqueryAlias: c_orders -06)----------Projection: COUNT(orders.o_orderkey) AS c_count -07)------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] +06)----------Projection: count(orders.o_orderkey) AS c_count +07)------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[count(orders.o_orderkey)]] 08)--------------Projection: customer.c_custkey, orders.o_orderkey 09)----------------Left Join: customer.c_custkey = orders.o_custkey 10)------------------TableScan: customer projection=[c_custkey] @@ -57,13 +57,13 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=10 02)--SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 03)----SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[true] -04)------ProjectionExec: expr=[c_count@0 as c_count, COUNT(*)@1 as custdist] -05)--------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(*)] +04)------ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] +05)--------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] -09)----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] -10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] +08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[count(*)] +09)----------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] +10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] 13)------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 2b01980f0e6f..d568b2ca69e6 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -52,8 +52,8 @@ limit 10; logical_plan 01)Limit: skip=0, fetch=10 02)--Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 -03)----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt -04)------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] +03)----Projection: part.p_brand, part.p_type, part.p_size, count(alias1) AS supplier_cnt +04)------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[count(alias1)]] 05)--------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] 06)----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey 07)------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size @@ -69,11 +69,11 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=10 02)--SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 03)----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] -04)------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] -05)--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +04)------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] +05)--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +08)--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] 09)----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index b536dd281eca..74c1c2fa77d7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -59,8 +59,8 @@ order by ---- logical_plan 01)Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST -02)--Projection: supplier.s_name, COUNT(*) AS numwait -03)----Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Projection: supplier.s_name, count(*) AS numwait +03)----Aggregate: groupBy=[[supplier.s_name]], aggr=[[count(Int64(1)) AS count(*)]] 04)------Projection: supplier.s_name 05)--------LeftAnti Join: l1.l_orderkey = __correlated_sq_2.l_orderkey Filter: __correlated_sq_2.l_suppkey != l1.l_suppkey 06)----------LeftSemi Join: l1.l_orderkey = __correlated_sq_1.l_orderkey Filter: __correlated_sq_1.l_suppkey != l1.l_suppkey @@ -92,11 +92,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[s_name@0 as s_name, COUNT(*)@1 as numwait] -04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(*)] +03)----ProjectionExec: expr=[s_name@0 as s_name, count(*)@1 as numwait] +04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] +07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(*)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] 10)------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index d05666b2513c..5b39c2725a97 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -57,8 +57,8 @@ order by ---- logical_plan 01)Sort: custsale.cntrycode ASC NULLS LAST -02)--Projection: custsale.cntrycode, COUNT(*) AS numcust, sum(custsale.c_acctbal) AS totacctbal -03)----Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[COUNT(Int64(1)) AS COUNT(*), sum(custsale.c_acctbal)]] +02)--Projection: custsale.cntrycode, count(*) AS numcust, sum(custsale.c_acctbal) AS totacctbal +03)----Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[count(Int64(1)) AS count(*), sum(custsale.c_acctbal)]] 04)------SubqueryAlias: custsale 05)--------Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal 06)----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.AVG(customer.c_acctbal) @@ -76,11 +76,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] 02)--SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(*)@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] -04)------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), sum(custsale.c_acctbal)] +03)----ProjectionExec: expr=[cntrycode@0 as cntrycode, count(*)@1 as numcust, sum(custsale.c_acctbal)@2 as totacctbal] +04)------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[count(*), sum(custsale.c_acctbal)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), sum(custsale.c_acctbal)] +07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[count(*), sum(custsale.c_acctbal)] 08)--------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index e2a5b9c5f009..b5a40e5b62d1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -41,8 +41,8 @@ order by ---- logical_plan 01)Sort: orders.o_orderpriority ASC NULLS LAST -02)--Projection: orders.o_orderpriority, COUNT(*) AS order_count -03)----Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Projection: orders.o_orderpriority, count(*) AS order_count +03)----Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[count(Int64(1)) AS count(*)]] 04)------Projection: orders.o_orderpriority 05)--------LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey 06)----------Projection: orders.o_orderkey, orders.o_orderpriority @@ -55,11 +55,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] 02)--SortExec: expr=[o_orderpriority@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(*)@1 as order_count] -04)------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] +03)----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, count(*)@1 as order_count] +04)------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] +07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(*)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 36f024961875..7b91e97e4a3e 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -420,16 +420,16 @@ SELECT count(*) FROM ( ) GROUP BY name ---- logical_plan -01)Projection: COUNT(*) -02)--Aggregate: groupBy=[[t1.name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Projection: count(*) +02)--Aggregate: groupBy=[[t1.name]], aggr=[[count(Int64(1)) AS count(*)]] 03)----Union 04)------Aggregate: groupBy=[[t1.name]], aggr=[[]] 05)--------TableScan: t1 projection=[name] 06)------Aggregate: groupBy=[[t2.name]], aggr=[[]] 07)--------TableScan: t2 projection=[name] physical_plan -01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] -02)--AggregateExec: mode=SinglePartitioned, gby=[name@0 as name], aggr=[COUNT(*)] +01)ProjectionExec: expr=[count(*)@1 as count(*)] +02)--AggregateExec: mode=SinglePartitioned, gby=[name@0 as name], aggr=[count(*)] 03)----InterleaveExec 04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 @@ -565,8 +565,8 @@ select x, y from (select 1 as x , max(10) as y) b ---- logical_plan 01)Union -02)--Projection: COUNT(*) AS count, a.n -03)----Aggregate: groupBy=[[a.n]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Projection: count(*) AS count, a.n +03)----Aggregate: groupBy=[[a.n]], aggr=[[count(Int64(1)) AS count(*)]] 04)------SubqueryAlias: a 05)--------Projection: Int64(5) AS n 06)----------EmptyRelation @@ -577,11 +577,11 @@ logical_plan 11)----------EmptyRelation physical_plan 01)UnionExec -02)--ProjectionExec: expr=[COUNT(*)@1 as count, n@0 as n] -03)----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[COUNT(*)], ordering_mode=Sorted +02)--ProjectionExec: expr=[count(*)@1 as count, n@0 as n] +03)----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[count(*)], ordering_mode=Sorted 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)], ordering_mode=Sorted +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] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 77b839f3f77a..d63917a78e42 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1303,14 +1303,14 @@ EXPLAIN SELECT FROM aggregate_test_100 ---- logical_plan -01)Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING -02)--WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +01)Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +02)--WindowAggr: windowExpr=[[count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 03)----Projection: aggregate_test_100.c1, aggregate_test_100.c2, sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING 04)------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 05)--------TableScan: aggregate_test_100 projection=[c1, c2, c4] physical_plan -01)ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] -02)--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] +01)ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), is_causal: false }], mode=[Sorted] 03)----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=4096 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 @@ -1763,8 +1763,8 @@ EXPLAIN SELECT count(*) as global_count FROM ORDER BY c1 ) AS a ---- logical_plan -01)Projection: COUNT(*) AS global_count -02)--Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +01)Projection: count(*) AS global_count +02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 03)----SubqueryAlias: a 04)------Projection: 05)--------Sort: aggregate_test_100.c1 ASC NULLS LAST @@ -1773,10 +1773,10 @@ logical_plan 08)--------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") 09)----------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] physical_plan -01)ProjectionExec: expr=[COUNT(*)@0 as global_count] -02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +01)ProjectionExec: expr=[count(*)@0 as global_count] +02)--AggregateExec: mode=Final, gby=[], aggr=[count(*)] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +04)------AggregateExec: mode=Partial, gby=[], aggr=[count(*)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 06)----------ProjectionExec: expr=[] 07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] @@ -2573,22 +2573,22 @@ 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 -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]] +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]] 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)--GlobalLimitExec: skip=0, fetch=5 03)----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC], preserve_partitioning=[false] -04)------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] -05)--------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] -06)----------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] -07)------------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] -08)--------------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] +04)------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] +05)--------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] +06)----------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] +07)------------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] +08)--------------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] 09)----------------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] 10)------------------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 @@ -2737,18 +2737,18 @@ 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)--GlobalLimitExec: skip=0, fetch=5 03)----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST], preserve_partitioning=[false] -04)------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] -05)--------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] -06)----------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] +04)------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] +05)--------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] +06)----------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] 07)------------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] 08)--------------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 @@ -2838,17 +2838,17 @@ logical_plan 01)Projection: sum1, sum2, count1, count2 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 -04)------Projection: sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts -05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +04)------Projection: sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] 07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS __common_expr_1, annotated_data_infinite.ts, annotated_data_infinite.inc_col 08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] +04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] 06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] 07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] @@ -2885,17 +2885,17 @@ logical_plan 01)Projection: sum1, sum2, count1, count2 02)--Limit: skip=0, fetch=5 03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 -04)------Projection: sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts -05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +04)------Projection: sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts +05)--------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[sum(__common_expr_1 AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] 07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS __common_expr_1, annotated_data_infinite.ts, annotated_data_infinite.inc_col 08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] +04)------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] 06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, ts@0 as ts, inc_col@1 as inc_col] 07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] @@ -4134,13 +4134,13 @@ query TT EXPLAIN select count(*) over (partition by a order by a) from (select * from a where a = 1); ---- logical_plan -01)Projection: COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -02)--WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +01)Projection: count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[count(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 03)----Filter: a.a = Int64(1) 04)------TableScan: a projection=[a] physical_plan -01)ProjectionExec: expr=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -02)--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, 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] +01)ProjectionExec: expr=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, 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] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 93f197885c0a..a48da7b9a1a8 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -983,7 +983,7 @@ pub async fn from_substrait_agg_func( // try udaf first, then built-in aggr fn. if let Ok(fun) = ctx.udaf(function_name) { // deal with situation that count(*) got no arguments - if fun.name() == "COUNT" && args.is_empty() { + if fun.name() == "count" && args.is_empty() { args.push(Expr::Literal(ScalarValue::Int64(Some(1)))); } diff --git a/datafusion/substrait/tests/cases/consumer_integration.rs b/datafusion/substrait/tests/cases/consumer_integration.rs index e0151ecc3a4f..975cc150d79c 100644 --- a/datafusion/substrait/tests/cases/consumer_integration.rs +++ b/datafusion/substrait/tests/cases/consumer_integration.rs @@ -46,9 +46,9 @@ mod tests { let plan_str = format!("{:?}", plan); assert_eq!( plan_str, - "Projection: FILENAME_PLACEHOLDER_0.l_returnflag AS L_RETURNFLAG, FILENAME_PLACEHOLDER_0.l_linestatus AS L_LINESTATUS, sum(FILENAME_PLACEHOLDER_0.l_quantity) AS SUM_QTY, sum(FILENAME_PLACEHOLDER_0.l_extendedprice) AS SUM_BASE_PRICE, sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount) AS SUM_DISC_PRICE, sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount * Int32(1) + FILENAME_PLACEHOLDER_0.l_tax) AS SUM_CHARGE, AVG(FILENAME_PLACEHOLDER_0.l_quantity) AS AVG_QTY, AVG(FILENAME_PLACEHOLDER_0.l_extendedprice) AS AVG_PRICE, AVG(FILENAME_PLACEHOLDER_0.l_discount) AS AVG_DISC, COUNT(Int64(1)) AS COUNT_ORDER\ + "Projection: FILENAME_PLACEHOLDER_0.l_returnflag AS L_RETURNFLAG, FILENAME_PLACEHOLDER_0.l_linestatus AS L_LINESTATUS, sum(FILENAME_PLACEHOLDER_0.l_quantity) AS SUM_QTY, sum(FILENAME_PLACEHOLDER_0.l_extendedprice) AS SUM_BASE_PRICE, sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount) AS SUM_DISC_PRICE, sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount * Int32(1) + FILENAME_PLACEHOLDER_0.l_tax) AS SUM_CHARGE, AVG(FILENAME_PLACEHOLDER_0.l_quantity) AS AVG_QTY, AVG(FILENAME_PLACEHOLDER_0.l_extendedprice) AS AVG_PRICE, AVG(FILENAME_PLACEHOLDER_0.l_discount) AS AVG_DISC, count(Int64(1)) AS COUNT_ORDER\ \n Sort: FILENAME_PLACEHOLDER_0.l_returnflag ASC NULLS LAST, FILENAME_PLACEHOLDER_0.l_linestatus ASC NULLS LAST\ - \n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.l_returnflag, FILENAME_PLACEHOLDER_0.l_linestatus]], aggr=[[sum(FILENAME_PLACEHOLDER_0.l_quantity), sum(FILENAME_PLACEHOLDER_0.l_extendedprice), sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount), sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount * Int32(1) + FILENAME_PLACEHOLDER_0.l_tax), AVG(FILENAME_PLACEHOLDER_0.l_quantity), AVG(FILENAME_PLACEHOLDER_0.l_extendedprice), AVG(FILENAME_PLACEHOLDER_0.l_discount), COUNT(Int64(1))]]\ + \n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.l_returnflag, FILENAME_PLACEHOLDER_0.l_linestatus]], aggr=[[sum(FILENAME_PLACEHOLDER_0.l_quantity), sum(FILENAME_PLACEHOLDER_0.l_extendedprice), sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount), sum(FILENAME_PLACEHOLDER_0.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_0.l_discount * Int32(1) + FILENAME_PLACEHOLDER_0.l_tax), AVG(FILENAME_PLACEHOLDER_0.l_quantity), AVG(FILENAME_PLACEHOLDER_0.l_extendedprice), AVG(FILENAME_PLACEHOLDER_0.l_discount), count(Int64(1))]]\ \n Projection: FILENAME_PLACEHOLDER_0.l_returnflag, FILENAME_PLACEHOLDER_0.l_linestatus, FILENAME_PLACEHOLDER_0.l_quantity, FILENAME_PLACEHOLDER_0.l_extendedprice, FILENAME_PLACEHOLDER_0.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_0.l_discount), FILENAME_PLACEHOLDER_0.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_0.l_discount) * (CAST(Int32(1) AS Decimal128(19, 0)) + FILENAME_PLACEHOLDER_0.l_tax), FILENAME_PLACEHOLDER_0.l_discount\ \n Filter: FILENAME_PLACEHOLDER_0.l_shipdate <= Date32(\"1998-12-01\") - IntervalDayTime(\"IntervalDayTime { days: 120, milliseconds: 0 }\")\ \n TableScan: FILENAME_PLACEHOLDER_0 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]" diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 4e4fa45a15a6..d236d34c9df9 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -594,10 +594,10 @@ async fn roundtrip_union_all() -> Result<()> { #[tokio::test] async fn simple_intersect() -> Result<()> { - // Substrait treats both COUNT(*) and COUNT(1) the same + // Substrait treats both count(*) and count(1) the same assert_expected_plan( - "SELECT COUNT(*) FROM (SELECT data.a FROM data INTERSECT SELECT data2.a FROM data2);", - "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]]\ + "SELECT count(*) FROM (SELECT data.a FROM data INTERSECT SELECT data2.a FROM data2);", + "Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]]\ \n Projection: \ \n LeftSemi Join: data.a = data2.a\ \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ @@ -610,7 +610,7 @@ async fn simple_intersect() -> Result<()> { #[tokio::test] async fn simple_intersect_table_reuse() -> Result<()> { - roundtrip("SELECT COUNT(1) FROM (SELECT data.a FROM data INTERSECT SELECT data.a FROM data);").await + roundtrip("SELECT count(1) FROM (SELECT data.a FROM data INTERSECT SELECT data.a FROM data);").await } #[tokio::test] From accd75b49a0030e831ac3b76d66b7ff9be6b0030 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 22 Jun 2024 19:29:02 +0800 Subject: [PATCH 49/54] Minor: Move `function::Hint` to `datafusion-expr` crate to avoid physical-expr dependency for `datafusion-function` crate (#11061) * mv to expr Signed-off-by: jayzhan211 * upd lock Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion-cli/Cargo.lock | 170 ++++++++++++------ datafusion/expr/src/function.rs | 8 + datafusion/functions/Cargo.toml | 1 - .../functions/src/regex/regexpreplace.rs | 2 +- datafusion/functions/src/string/btrim.rs | 2 +- datafusion/functions/src/string/ltrim.rs | 2 +- datafusion/functions/src/string/rtrim.rs | 2 +- datafusion/functions/src/utils.rs | 2 +- datafusion/physical-expr/src/functions.rs | 10 +- 9 files changed, 127 insertions(+), 72 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index c5b34df4f1cf..995a49d29e7a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -387,7 +387,7 @@ checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -714,9 +714,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.72" +version = "0.3.73" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17c6a35df3749d2e8bb1b7b21a976d82b15548788d2735b9d82f329268f71a11" +checksum = "5cc23269a4f8976d0a4d2e7109211a419fe30e8d88d677cd60b6bc79c5732e0a" dependencies = [ "addr2line", "cc", @@ -1099,7 +1099,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -1278,7 +1278,6 @@ dependencies = [ "datafusion-common", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "hashbrown 0.14.5", "hex", "itertools", @@ -1687,7 +1686,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -1911,12 +1910,12 @@ dependencies = [ [[package]] name = "http-body-util" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0475f8b2ac86659c21b64320d5d653f9efe42acd2a4e560073ec61a155a34f1d" +checksum = "793429d76616a256bcb62c2a2ec2bed781c8307e797e2598c50010f2bee2544f" dependencies = [ "bytes", - "futures-core", + "futures-util", "http 1.1.0", "http-body 1.0.0", "pin-project-lite", @@ -1924,9 +1923,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.8.0" +version = "1.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" +checksum = "0fcc0b4a115bf80b728eb8ea024ad5bd707b615bfed49e0665b6e0f86fd082d9" [[package]] name = "httpdate" @@ -2001,18 +2000,19 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.26.0" +version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0bea761b46ae2b24eb4aef630d8d1c398157b6fc29e6350ecf090a0b70c952c" +checksum = "5ee4be2c948921a1a5320b629c4193916ed787a7f7f293fd3f7f5a6c9de74155" dependencies = [ "futures-util", "http 1.1.0", "hyper 1.3.1", "hyper-util", - "rustls 0.22.4", + "rustls 0.23.10", + "rustls-native-certs 0.7.0", "rustls-pki-types", "tokio", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "tower-service", ] @@ -2148,9 +2148,9 @@ dependencies = [ [[package]] name = "lazy_static" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" [[package]] name = "lexical-core" @@ -2326,9 +2326,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.2" +version = "2.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" [[package]] name = "mimalloc" @@ -2347,9 +2347,9 @@ checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" [[package]] name = "miniz_oxide" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87dfd01fe195c66b572b37921ad8803d010623c0aca821bea2302239d155cdae" +checksum = "b8a240ddb74feaf34a79a7add65a741f3167852fba007066dcac1ca548d89c08" dependencies = [ "adler", ] @@ -2483,9 +2483,9 @@ dependencies = [ [[package]] name = "object" -version = "0.35.0" +version = "0.36.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8ec7ab813848ba4522158d5517a6093db1ded27575b070f4177b8d12b41db5e" +checksum = "576dfe1fc8f9df304abb159d767a29d0476f7750fbf8aa7ad07816004a207434" dependencies = [ "memchr", ] @@ -2699,7 +2699,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -2788,9 +2788,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.85" +version = "1.0.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" +checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" dependencies = [ "unicode-ident", ] @@ -2811,6 +2811,53 @@ dependencies = [ "serde", ] +[[package]] +name = "quinn" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4ceeeeabace7857413798eb1ffa1e9c905a9946a57d81fb69b4b71c4d8eb3ad" +dependencies = [ + "bytes", + "pin-project-lite", + "quinn-proto", + "quinn-udp", + "rustc-hash", + "rustls 0.23.10", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "quinn-proto" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddf517c03a109db8100448a4be38d498df8a210a99fe0e1b9eaf39e78c640efe" +dependencies = [ + "bytes", + "rand", + "ring 0.17.8", + "rustc-hash", + "rustls 0.23.10", + "slab", + "thiserror", + "tinyvec", + "tracing", +] + +[[package]] +name = "quinn-udp" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9096629c45860fc7fb143e125eb826b5e721e10be3263160c7d60ca832cf8c46" +dependencies = [ + "libc", + "once_cell", + "socket2", + "tracing", + "windows-sys 0.52.0", +] + [[package]] name = "quote" version = "1.0.36" @@ -2862,9 +2909,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.1" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e" +checksum = "c82cf8cff14456045f55ec4241383baeff27af886adb72ffb2162f99911de0fd" dependencies = [ "bitflags 2.5.0", ] @@ -2917,9 +2964,9 @@ checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" [[package]] name = "reqwest" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "566cafdd92868e0939d3fb961bd0dc25fcfaaed179291093b3d43e6b3150ea10" +checksum = "c7d6d2a27d57148378eb5e111173f4276ad26340ecc5c49a4a2152167a2d6a37" dependencies = [ "base64 0.22.1", "bytes", @@ -2930,7 +2977,7 @@ dependencies = [ "http-body 1.0.0", "http-body-util", "hyper 1.3.1", - "hyper-rustls 0.26.0", + "hyper-rustls 0.27.2", "hyper-util", "ipnet", "js-sys", @@ -2939,7 +2986,8 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.22.4", + "quinn", + "rustls 0.23.10", "rustls-native-certs 0.7.0", "rustls-pemfile 2.1.2", "rustls-pki-types", @@ -2948,7 +2996,7 @@ dependencies = [ "serde_urlencoded", "sync_wrapper", "tokio", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "tokio-util", "tower-service", "url", @@ -3027,6 +3075,12 @@ version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + [[package]] name = "rustc_version" version = "0.4.0" @@ -3063,11 +3117,11 @@ dependencies = [ [[package]] name = "rustls" -version = "0.22.4" +version = "0.23.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf4ef73721ac7bcd79b2b315da7779d8fc09718c6b3d2d1b2d94850eb8c18432" +checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402" dependencies = [ - "log", + "once_cell", "ring 0.17.8", "rustls-pki-types", "rustls-webpki", @@ -3257,7 +3311,7 @@ checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3392,7 +3446,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3438,7 +3492,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3451,14 +3505,14 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] name = "subtle" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" +checksum = "0d0208408ba0c3df17ed26eb06992cb1a1268d41b2c0e12e65203fbe3972cee5" [[package]] name = "syn" @@ -3473,9 +3527,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.66" +version = "2.0.67" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5" +checksum = "ff8655ed1d86f3af4ee3fd3263786bc14245ad17c4c7e85ba7187fb3ae028c90" dependencies = [ "proc-macro2", "quote", @@ -3484,9 +3538,9 @@ dependencies = [ [[package]] name = "sync_wrapper" -version = "0.1.2" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" +checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" [[package]] name = "tempfile" @@ -3538,7 +3592,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3633,7 +3687,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3649,11 +3703,11 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.25.0" +version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "775e0c0f0adb3a2f22a00c4745d728b479985fc15ee7ca6a2608388c5569860f" +checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" dependencies = [ - "rustls 0.22.4", + "rustls 0.23.10", "rustls-pki-types", "tokio", ] @@ -3730,7 +3784,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3775,7 +3829,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] @@ -3831,9 +3885,9 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.5.0" +version = "2.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" +checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c" dependencies = [ "form_urlencoded", "idna", @@ -3929,7 +3983,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", "wasm-bindgen-shared", ] @@ -3963,7 +4017,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4228,7 +4282,7 @@ checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.67", ] [[package]] diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 169436145aae..73ab51494de6 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -23,6 +23,14 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use std::sync::Arc; +#[derive(Debug, Clone, Copy)] +pub enum Hint { + /// Indicates the argument needs to be padded if it is scalar + Pad, + /// Indicates the argument can be converted to an array of length 1 + AcceptsSingular, +} + /// Scalar function /// /// The Fn param is the wrapped function but be aware that the function will diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index 20d6cbc37459..884a66724c91 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -73,7 +73,6 @@ chrono = { workspace = true } datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr = { workspace = true, default-features = true } hashbrown = { workspace = true, optional = true } hex = { version = "0.4", optional = true } itertools = { workspace = true } diff --git a/datafusion/functions/src/regex/regexpreplace.rs b/datafusion/functions/src/regex/regexpreplace.rs index 4e21883c9752..201eebde22bb 100644 --- a/datafusion/functions/src/regex/regexpreplace.rs +++ b/datafusion/functions/src/regex/regexpreplace.rs @@ -28,10 +28,10 @@ use datafusion_common::ScalarValue; use datafusion_common::{ cast::as_generic_string_array, internal_err, DataFusionError, Result, }; +use datafusion_expr::function::Hint; use datafusion_expr::ColumnarValue; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; -use datafusion_physical_expr::functions::Hint; use regex::Regex; use std::any::Any; use std::collections::HashMap; diff --git a/datafusion/functions/src/string/btrim.rs b/datafusion/functions/src/string/btrim.rs index 97b54a194a27..349928d09664 100644 --- a/datafusion/functions/src/string/btrim.rs +++ b/datafusion/functions/src/string/btrim.rs @@ -21,10 +21,10 @@ use std::any::Any; use arrow::datatypes::DataType; use datafusion_common::{exec_err, Result}; +use datafusion_expr::function::Hint; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ColumnarValue, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; -use datafusion_physical_expr::functions::Hint; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; diff --git a/datafusion/functions/src/string/ltrim.rs b/datafusion/functions/src/string/ltrim.rs index ef05a2cb2a13..de14bbaa2bcf 100644 --- a/datafusion/functions/src/string/ltrim.rs +++ b/datafusion/functions/src/string/ltrim.rs @@ -21,10 +21,10 @@ use arrow::array::{ArrayRef, OffsetSizeTrait}; use arrow::datatypes::DataType; use datafusion_common::{exec_err, Result}; +use datafusion_expr::function::Hint; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ColumnarValue, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; -use datafusion_physical_expr::functions::Hint; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; diff --git a/datafusion/functions/src/string/rtrim.rs b/datafusion/functions/src/string/rtrim.rs index 2e39080e226b..2d29b50cb173 100644 --- a/datafusion/functions/src/string/rtrim.rs +++ b/datafusion/functions/src/string/rtrim.rs @@ -21,10 +21,10 @@ use std::any::Any; use arrow::datatypes::DataType; use datafusion_common::{exec_err, Result}; +use datafusion_expr::function::Hint; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ColumnarValue, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; -use datafusion_physical_expr::functions::Hint; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; diff --git a/datafusion/functions/src/utils.rs b/datafusion/functions/src/utils.rs index d14844c4a445..393dcc456a88 100644 --- a/datafusion/functions/src/utils.rs +++ b/datafusion/functions/src/utils.rs @@ -18,8 +18,8 @@ use arrow::array::ArrayRef; use arrow::datatypes::DataType; use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::function::Hint; use datafusion_expr::{ColumnarValue, ScalarFunctionImplementation}; -use datafusion_physical_expr::functions::Hint; use std::sync::Arc; /// Creates a function to identify the optimal return type of a string function given diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 9c7d6d09349d..e33c28df1988 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -38,14 +38,8 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{ColumnarValue, ScalarFunctionImplementation}; pub use crate::scalar_function::create_physical_expr; - -#[derive(Debug, Clone, Copy)] -pub enum Hint { - /// Indicates the argument needs to be padded if it is scalar - Pad, - /// Indicates the argument can be converted to an array of length 1 - AcceptsSingular, -} +// For backward compatibility +pub use datafusion_expr::function::Hint; #[deprecated(since = "36.0.0", note = "Use ColumarValue::values_to_arrays instead")] pub fn columnar_values_to_array(args: &[ColumnarValue]) -> Result> { From 81611ad2c5441721b0be6fcd08d915e4f1efd24a Mon Sep 17 00:00:00 2001 From: Kevin Su Date: Sat, 22 Jun 2024 19:32:41 +0800 Subject: [PATCH 50/54] Support to unparse ScalarValue::TimestampMillisecond to String (#11046) * wip Signed-off-by: Kevin Su * add a test Signed-off-by: Kevin Su --------- Signed-off-by: Kevin Su --- datafusion/sql/src/unparser/expr.rs | 58 ++++++++++++++++++++++++----- 1 file changed, 49 insertions(+), 9 deletions(-) diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 7db6956e3244..0aefefc1f007 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -19,7 +19,9 @@ use arrow::util::display::array_value_to_string; use core::fmt; use std::{fmt::Display, vec}; -use arrow_array::{Date32Array, Date64Array, TimestampNanosecondArray}; +use arrow_array::{ + Date32Array, Date64Array, TimestampMillisecondArray, TimestampNanosecondArray, +}; use arrow_schema::DataType; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ @@ -647,6 +649,15 @@ impl Unparser<'_> { } } + fn timestamp_string_to_sql(&self, ts: String) -> Result { + Ok(ast::Expr::Cast { + kind: ast::CastKind::Cast, + expr: Box::new(ast::Expr::Value(SingleQuotedString(ts))), + data_type: ast::DataType::Timestamp(None, TimezoneInfo::None), + format: None, + }) + } + /// DataFusion ScalarValues sometimes require a ast::Expr to construct. /// For example ScalarValue::Date32(d) corresponds to the ast::Expr CAST('datestr' as DATE) fn scalar_to_sql(&self, v: &ScalarValue) -> Result { @@ -808,8 +819,31 @@ impl Unparser<'_> { ScalarValue::TimestampSecond(None, _) => { Ok(ast::Expr::Value(ast::Value::Null)) } - ScalarValue::TimestampMillisecond(Some(_ts), _) => { - not_impl_err!("Unsupported scalar: {v:?}") + ScalarValue::TimestampMillisecond(Some(_ts), tz) => { + let result = if let Some(tz) = tz { + v.to_array()? + .as_any() + .downcast_ref::() + .ok_or(internal_datafusion_err!( + "Unable to downcast to TimestampMillisecond from TimestampMillisecond scalar" + ))? + .value_as_datetime_with_tz(0, tz.parse()?) + .ok_or(internal_datafusion_err!( + "Unable to convert TimestampMillisecond to DateTime" + ))?.to_string() + } else { + v.to_array()? + .as_any() + .downcast_ref::() + .ok_or(internal_datafusion_err!( + "Unable to downcast to TimestampMillisecond from TimestampMillisecond scalar" + ))? + .value_as_datetime(0) + .ok_or(internal_datafusion_err!( + "Unable to convert TimestampMillisecond to NaiveDateTime" + ))?.to_string() + }; + self.timestamp_string_to_sql(result) } ScalarValue::TimestampMillisecond(None, _) => { Ok(ast::Expr::Value(ast::Value::Null)) @@ -844,12 +878,7 @@ impl Unparser<'_> { "Unable to convert TimestampNanosecond to NaiveDateTime" ))?.to_string() }; - Ok(ast::Expr::Cast { - kind: ast::CastKind::Cast, - expr: Box::new(ast::Expr::Value(SingleQuotedString(result))), - data_type: ast::DataType::Timestamp(None, TimezoneInfo::None), - format: None, - }) + self.timestamp_string_to_sql(result) } ScalarValue::TimestampNanosecond(None, _) => { Ok(ast::Expr::Value(ast::Value::Null)) @@ -1180,6 +1209,17 @@ mod tests { Expr::Literal(ScalarValue::Date32(Some(-1))), r#"CAST('1969-12-31' AS DATE)"#, ), + ( + Expr::Literal(ScalarValue::TimestampMillisecond(Some(10001), None)), + r#"CAST('1970-01-01 00:00:10.001' AS TIMESTAMP)"#, + ), + ( + Expr::Literal(ScalarValue::TimestampMillisecond( + Some(10001), + Some("+08:00".into()), + )), + r#"CAST('1970-01-01 08:00:10.001 +08:00' AS TIMESTAMP)"#, + ), ( Expr::Literal(ScalarValue::TimestampNanosecond(Some(10001), None)), r#"CAST('1970-01-01 00:00:00.000010001' AS TIMESTAMP)"#, From 8a98307f2afaae3eb5c3293fe4454af4d472748e Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 22 Jun 2024 19:34:50 +0800 Subject: [PATCH 51/54] support to unparse interval to string (#11065) --- datafusion/expr/src/expr_fn.rs | 14 ++++++++- datafusion/sql/src/unparser/expr.rs | 44 ++++++++++++++++++----------- 2 files changed, 41 insertions(+), 17 deletions(-) diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index a87412ee6356..deac20555771 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -31,7 +31,9 @@ use crate::{ Signature, Volatility, }; use crate::{AggregateUDFImpl, ColumnarValue, ScalarUDFImpl, WindowUDF, WindowUDFImpl}; -use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; +use arrow::compute::kernels::cast_utils::{ + parse_interval_day_time, parse_interval_month_day_nano, parse_interval_year_month, +}; use arrow::datatypes::{DataType, Field}; use datafusion_common::{Column, Result, ScalarValue}; use std::any::Any; @@ -671,6 +673,16 @@ impl WindowUDFImpl for SimpleWindowUDF { } } +pub fn interval_year_month_lit(value: &str) -> Expr { + let interval = parse_interval_year_month(value).ok(); + Expr::Literal(ScalarValue::IntervalYearMonth(interval)) +} + +pub fn interval_datetime_lit(value: &str) -> Expr { + let interval = parse_interval_day_time(value).ok(); + Expr::Literal(ScalarValue::IntervalDayTime(interval)) +} + pub fn interval_month_day_nano_lit(value: &str) -> Expr { let interval = parse_interval_month_day_nano(value).ok(); Expr::Literal(ScalarValue::IntervalMonthDayNano(interval)) diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 0aefefc1f007..3ad8759368a2 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -883,21 +883,13 @@ impl Unparser<'_> { ScalarValue::TimestampNanosecond(None, _) => { Ok(ast::Expr::Value(ast::Value::Null)) } - ScalarValue::IntervalYearMonth(Some(_i)) => { - not_impl_err!("Unsupported scalar: {v:?}") - } - ScalarValue::IntervalYearMonth(None) => { - Ok(ast::Expr::Value(ast::Value::Null)) - } - ScalarValue::IntervalDayTime(Some(_i)) => { - not_impl_err!("Unsupported scalar: {v:?}") - } - ScalarValue::IntervalDayTime(None) => Ok(ast::Expr::Value(ast::Value::Null)), - ScalarValue::IntervalMonthDayNano(Some(_i)) => { + ScalarValue::IntervalYearMonth(Some(_)) + | ScalarValue::IntervalDayTime(Some(_)) + | ScalarValue::IntervalMonthDayNano(Some(_)) => { let wrap_array = v.to_array()?; let Some(result) = array_value_to_string(&wrap_array, 0).ok() else { return internal_err!( - "Unable to convert IntervalMonthDayNano to string" + "Unable to convert interval scalar value to string" ); }; let interval = Interval { @@ -911,6 +903,10 @@ impl Unparser<'_> { }; Ok(ast::Expr::Interval(interval)) } + ScalarValue::IntervalYearMonth(None) => { + Ok(ast::Expr::Value(ast::Value::Null)) + } + ScalarValue::IntervalDayTime(None) => Ok(ast::Expr::Value(ast::Value::Null)), ScalarValue::IntervalMonthDayNano(None) => { Ok(ast::Expr::Value(ast::Value::Null)) } @@ -1044,10 +1040,10 @@ mod tests { use arrow_schema::DataType::Int8; use datafusion_common::TableReference; use datafusion_expr::{ - case, col, cube, exists, grouping_set, lit, not, not_exists, out_ref_col, - placeholder, rollup, table_scan, try_cast, when, wildcard, ColumnarValue, - ScalarUDF, ScalarUDFImpl, Signature, Volatility, WindowFrame, - WindowFunctionDefinition, + case, col, cube, exists, grouping_set, interval_datetime_lit, + interval_year_month_lit, lit, not, not_exists, out_ref_col, placeholder, rollup, + table_scan, try_cast, when, wildcard, ColumnarValue, ScalarUDF, ScalarUDFImpl, + Signature, Volatility, WindowFrame, WindowFunctionDefinition, }; use datafusion_expr::{interval_month_day_nano_lit, AggregateExt}; use datafusion_functions_aggregate::count::count_udaf; @@ -1385,6 +1381,22 @@ mod tests { .sub(interval_month_day_nano_lit("1 DAY")), r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' - INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, ), + ( + interval_datetime_lit("10 DAY 1 HOUR 10 MINUTE 20 SECOND"), + r#"INTERVAL '0 YEARS 0 MONS 10 DAYS 1 HOURS 10 MINS 20.000 SECS'"#, + ), + ( + interval_datetime_lit("10 DAY 1.5 HOUR 10 MINUTE 20 SECOND"), + r#"INTERVAL '0 YEARS 0 MONS 10 DAYS 1 HOURS 40 MINS 20.000 SECS'"#, + ), + ( + interval_year_month_lit("1 YEAR 1 MONTH"), + r#"INTERVAL '1 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.00 SECS'"#, + ), + ( + interval_year_month_lit("1.5 YEAR 1 MONTH"), + r#"INTERVAL '1 YEARS 7 MONS 0 DAYS 0 HOURS 0 MINS 0.00 SECS'"#, + ), ]; for (expr, expected) in tests { From a22423d5265434cfb9603128998665489e318ccc Mon Sep 17 00:00:00 2001 From: Kevin Su Date: Sat, 22 Jun 2024 20:02:22 +0800 Subject: [PATCH 52/54] feat: Add method to add analyzer rules to SessionContext (#10849) * feat: Add method to add analyzer rules to SessionContext Signed-off-by: Kevin Su * Add a test Signed-off-by: Kevin Su * Add analyze_plan Signed-off-by: Kevin Su * update test Signed-off-by: Kevin Su --------- Signed-off-by: Kevin Su Co-authored-by: Andrew Lamb --- datafusion/core/src/execution/context/mod.rs | 10 ++ .../core/src/execution/session_state.rs | 4 +- .../tests/user_defined/user_defined_plan.rs | 99 ++++++++++++++++++- 3 files changed, 107 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 6fa83d3d931e..c44e9742607e 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -75,6 +75,7 @@ use url::Url; pub use datafusion_execution::config::SessionConfig; pub use datafusion_execution::TaskContext; pub use datafusion_expr::execution_props::ExecutionProps; +use datafusion_optimizer::AnalyzerRule; mod avro; mod csv; @@ -331,6 +332,15 @@ impl SessionContext { self } + /// Adds an analyzer rule to the `SessionState` in the current `SessionContext`. + pub fn add_analyzer_rule( + self, + analyzer_rule: Arc, + ) -> Self { + self.state.write().add_analyzer_rule(analyzer_rule); + self + } + /// Registers an [`ObjectStore`] to be used with a specific URL prefix. /// /// See [`RuntimeEnv::register_object_store`] for more details. diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 1df77a1f9e0b..e9441a89cd5f 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -384,9 +384,9 @@ impl SessionState { /// Add `analyzer_rule` to the end of the list of /// [`AnalyzerRule`]s used to rewrite queries. pub fn add_analyzer_rule( - mut self, + &mut self, analyzer_rule: Arc, - ) -> Self { + ) -> &Self { self.analyzer.rules.push(analyzer_rule); self } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index ebf907c5e2c0..c5654ded888a 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -92,8 +92,12 @@ use datafusion::{ }; use async_trait::async_trait; -use datafusion_common::tree_node::Transformed; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::ScalarValue; +use datafusion_expr::Projection; use datafusion_optimizer::optimizer::ApplyOrder; +use datafusion_optimizer::AnalyzerRule; use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches @@ -132,11 +136,13 @@ async fn setup_table_without_schemas(mut ctx: SessionContext) -> Result Result<()> { @@ -164,6 +170,34 @@ async fn run_and_compare_query(mut ctx: SessionContext, description: &str) -> Re Ok(()) } +// 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, + description: &str, +) -> Result<()> { + let expected = vec![ + "+------------+--------------------------+", + "| UInt64(42) | arrow_typeof(UInt64(42)) |", + "+------------+--------------------------+", + "| 42 | UInt64 |", + "+------------+--------------------------+", + ]; + + let s = exec_sql(&mut ctx, QUERY2).await?; + let actual = s.lines().collect::>(); + + assert_eq!( + expected, + actual, + "output mismatch for {}. Expectedn\n{}Actual:\n{}", + description, + expected.join("\n"), + s + ); + Ok(()) +} + // Run the query using the specified execution context and compare it // to the known result async fn run_and_compare_query_with_auto_schemas( @@ -208,6 +242,13 @@ async fn normal_query() -> Result<()> { run_and_compare_query(ctx, "Default context").await } +#[tokio::test] +// Run the query using default planners, optimizer and custom analyzer rule +async fn normal_query_with_analyzer() -> Result<()> { + let ctx = SessionContext::new().add_analyzer_rule(Arc::new(MyAnalyzerRule {})); + run_and_compare_query_with_analyzer_rule(ctx, "MyAnalyzerRule").await +} + #[tokio::test] // Run the query using topk optimization async fn topk_query() -> Result<()> { @@ -248,9 +289,10 @@ async fn topk_plan() -> Result<()> { fn make_topk_context() -> SessionContext { let config = SessionConfig::new().with_target_partitions(48); let runtime = Arc::new(RuntimeEnv::default()); - let state = SessionState::new_with_config_rt(config, runtime) + let mut state = SessionState::new_with_config_rt(config, runtime) .with_query_planner(Arc::new(TopKQueryPlanner {})) .add_optimizer_rule(Arc::new(TopKOptimizerRule {})); + state.add_analyzer_rule(Arc::new(MyAnalyzerRule {})); SessionContext::new_with_state(state) } @@ -633,3 +675,52 @@ impl RecordBatchStream for TopKReader { self.input.schema() } } + +struct MyAnalyzerRule {} + +impl AnalyzerRule for MyAnalyzerRule { + fn analyze(&self, plan: LogicalPlan, _config: &ConfigOptions) -> Result { + Self::analyze_plan(plan) + } + + fn name(&self) -> &str { + "my_analyzer_rule" + } +} + +impl MyAnalyzerRule { + fn analyze_plan(plan: LogicalPlan) -> Result { + plan.transform(|plan| { + Ok(match plan { + LogicalPlan::Projection(projection) => { + let expr = Self::analyze_expr(projection.expr.clone())?; + Transformed::yes(LogicalPlan::Projection(Projection::try_new( + expr, + projection.input, + )?)) + } + _ => Transformed::no(plan), + }) + }) + .data() + } + + fn analyze_expr(expr: Vec) -> Result> { + expr.into_iter() + .map(|e| { + e.transform(|e| { + Ok(match e { + Expr::Literal(ScalarValue::Int64(i)) => { + // transform to UInt64 + Transformed::yes(Expr::Literal(ScalarValue::UInt64( + i.map(|i| i as u64), + ))) + } + _ => Transformed::no(e), + }) + }) + .data() + }) + .collect() + } +} From 6c0e4fb5d9ac7a0a2f2b91f8b88d21f0bc0b4424 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Sat, 22 Jun 2024 05:05:13 -0700 Subject: [PATCH 53/54] SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join (#11041) * Fix: Sort Merge Join crashes on TPCH Q21 * Fix LeftAnti SMJ join when the join filter is set * rm dbg * Minor: disable fuzz test to avoid CI spontaneous failures * Minor: disable fuzz test to avoid CI spontaneous failures * Fix: Sort Merge Join crashes on TPCH Q21 * Fix LeftAnti SMJ join when the join filter is set * rm dbg * Minor: disable fuzz test to avoid CI spontaneous failures * Minor: disable fuzz test to avoid CI spontaneous failures * Minor: Add routine to debug join fuzz tests * Minor: Add routine to debug join fuzz tests * Minor: Add routine to debug join fuzz tests * Minor: Add routine to debug join fuzz tests * Minor: Add routine to debug join fuzz tests * SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join * SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join * SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 104 +++++++++++++++--- .../src/joins/sort_merge_join.rs | 24 +++- 2 files changed, 110 insertions(+), 18 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 5fdf02079496..17dbf3a0ff28 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -215,10 +215,6 @@ async fn test_semi_join_1k() { .await } -// The test is flaky -// https://github.com/apache/datafusion/issues/10886 -// SMJ produces 1 more row in the output -#[ignore] #[tokio::test] async fn test_semi_join_1k_filtered() { JoinFuzzTestCase::new( @@ -442,18 +438,45 @@ impl JoinFuzzTestCase { if debug { println!("The debug is ON. Input data will be saved"); - let out_dir_name = &format!("fuzz_test_debug_batch_size_{batch_size}"); - Self::save_as_parquet(&self.input1, out_dir_name, "input1"); - Self::save_as_parquet(&self.input2, out_dir_name, "input2"); + let fuzz_debug = "fuzz_test_debug"; + std::fs::remove_dir_all(fuzz_debug).unwrap_or(()); + std::fs::create_dir_all(fuzz_debug).unwrap(); + let out_dir_name = &format!("{fuzz_debug}/batch_size_{batch_size}"); + Self::save_partitioned_batches_as_parquet( + &self.input1, + out_dir_name, + "input1", + ); + Self::save_partitioned_batches_as_parquet( + &self.input2, + out_dir_name, + "input2", + ); if join_tests.contains(&JoinTestType::NljHj) { - Self::save_as_parquet(&nlj_collected, out_dir_name, "nlj"); - Self::save_as_parquet(&hj_collected, out_dir_name, "hj"); + Self::save_partitioned_batches_as_parquet( + &nlj_collected, + out_dir_name, + "nlj", + ); + Self::save_partitioned_batches_as_parquet( + &hj_collected, + out_dir_name, + "hj", + ); } if join_tests.contains(&JoinTestType::HjSmj) { - Self::save_as_parquet(&hj_collected, out_dir_name, "hj"); - Self::save_as_parquet(&smj_collected, out_dir_name, "smj"); + Self::save_partitioned_batches_as_parquet( + &hj_collected, + out_dir_name, + "hj", + ); + Self::save_partitioned_batches_as_parquet( + &smj_collected, + out_dir_name, + "smj", + ); } } @@ -527,11 +550,26 @@ impl JoinFuzzTestCase { /// as a parquet files preserving partitioning. /// Once the data is saved it is possible to run a custom test on top of the saved data and debug /// + /// #[tokio::test] + /// async fn test1() { + /// let left: Vec = JoinFuzzTestCase::load_partitioned_batches_from_parquet("fuzz_test_debug/batch_size_2/input1").await.unwrap(); + /// let right: Vec = JoinFuzzTestCase::load_partitioned_batches_from_parquet("fuzz_test_debug/batch_size_2/input2").await.unwrap(); + /// + /// JoinFuzzTestCase::new( + /// left, + /// right, + /// JoinType::LeftSemi, + /// Some(Box::new(col_lt_col_filter)), + /// ) + /// .run_test(&[JoinTestType::HjSmj], false) + /// .await + /// } + /// /// let ctx: SessionContext = SessionContext::new(); /// let df = ctx /// .read_parquet( /// "/tmp/input1/*.parquet", - /// ParquetReadOptions::default(), + /// datafusion::prelude::ParquetReadOptions::default(), /// ) /// .await /// .unwrap(); @@ -540,7 +578,7 @@ impl JoinFuzzTestCase { /// let df = ctx /// .read_parquet( /// "/tmp/input2/*.parquet", - /// ParquetReadOptions::default(), + /// datafusion::prelude::ParquetReadOptions::default(), /// ) /// .await /// .unwrap(); @@ -554,8 +592,11 @@ impl JoinFuzzTestCase { /// ) /// .run_test() /// .await - /// } - fn save_as_parquet(input: &[RecordBatch], output_dir: &str, out_name: &str) { + fn save_partitioned_batches_as_parquet( + input: &[RecordBatch], + output_dir: &str, + out_name: &str, + ) { let out_path = &format!("{output_dir}/{out_name}"); std::fs::remove_dir_all(out_path).unwrap_or(()); std::fs::create_dir_all(out_path).unwrap(); @@ -576,6 +617,39 @@ impl JoinFuzzTestCase { println!("The data {out_name} saved as parquet into {out_path}"); } + + /// Read parquet files preserving partitions, i.e. 1 file -> 1 partition + /// Files can be of different sizes + /// The method can be useful to read partitions have been saved by `save_partitioned_batches_as_parquet` + /// for test debugging purposes + #[allow(dead_code)] + async fn load_partitioned_batches_from_parquet( + dir: &str, + ) -> std::io::Result> { + let ctx: SessionContext = SessionContext::new(); + let mut batches: Vec = vec![]; + + for entry in std::fs::read_dir(dir)? { + let entry = entry?; + let path = entry.path(); + + if path.is_file() { + let mut batch = ctx + .read_parquet( + path.to_str().unwrap(), + datafusion::prelude::ParquetReadOptions::default(), + ) + .await + .unwrap() + .collect() + .await + .unwrap(); + + batches.append(&mut batch); + } + } + Ok(batches) + } } /// Return randomly sized record batches with: diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 420fab51da39..91b2151d32e7 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -1532,17 +1532,21 @@ fn get_filtered_join_mask( for i in 0..streamed_indices_length { // LeftSemi respects only first true values for specific streaming index, // others true values for the same index must be false - if mask.value(i) && !seen_as_true { + let streamed_idx = streamed_indices.value(i); + if mask.value(i) + && !seen_as_true + && !matched_indices.contains(&streamed_idx) + { seen_as_true = true; corrected_mask.append_value(true); - filter_matched_indices.push(streamed_indices.value(i)); + filter_matched_indices.push(streamed_idx); } else { corrected_mask.append_value(false); } // if switched to next streaming index(e.g. from 0 to 1, or from 1 to 2), we reset seen_as_true flag if i < streamed_indices_length - 1 - && streamed_indices.value(i) != streamed_indices.value(i + 1) + && streamed_idx != streamed_indices.value(i + 1) { seen_as_true = false; } @@ -2940,6 +2944,20 @@ mod tests { )) ); + assert_eq!( + get_filtered_join_mask( + LeftSemi, + &UInt64Array::from(vec![0, 0, 0, 1, 1, 1]), + &BooleanArray::from(vec![true, false, false, false, false, true]), + &HashSet::from_iter(vec![1]), + &0, + ), + Some(( + BooleanArray::from(vec![true, false, false, false, false, false]), + vec![0] + )) + ); + Ok(()) } From ea46e82088ceb27f85de5a994495649ef0bd4747 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 22 Jun 2024 08:17:07 -0400 Subject: [PATCH 54/54] Add `advanced_parquet_index.rs` example of index in into parquet files (#10701) * Add `advanced_parquet_index.rs` example of indexing into parquet files * pre-load page index * fix comment * Apply suggestions from code review Thank you @Weijun-H Co-authored-by: Alex Huang * Add ASCII ART * Update datafusion-examples/README.md Co-authored-by: Alex Huang * Update datafusion-examples/examples/advanced_parquet_index.rs Co-authored-by: Alex Huang * Improve / clarify comments based on review * Add page index caveat --------- Co-authored-by: Alex Huang --- datafusion-examples/README.md | 1 + .../examples/advanced_parquet_index.rs | 664 ++++++++++++++++++ datafusion/common/src/column.rs | 7 + datafusion/common/src/config.rs | 7 + .../physical_plan/parquet/access_plan.rs | 5 + .../datasource/physical_plan/parquet/mod.rs | 4 +- .../physical_plan/parquet/row_groups.rs | 2 +- .../core/src/physical_optimizer/pruning.rs | 10 +- 8 files changed, 695 insertions(+), 5 deletions(-) create mode 100644 datafusion-examples/examples/advanced_parquet_index.rs diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 6150c551c900..0eef9091b2bc 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -45,6 +45,7 @@ cargo run --example csv_sql - [`advanced_udaf.rs`](examples/advanced_udaf.rs): Define and invoke a more complicated User Defined Aggregate Function (UDAF) - [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF) - [`advanced_udwf.rs`](examples/advanced_udwf.rs): Define and invoke a more complicated User Defined Window Function (UDWF) +- [`advanced_parquet_index.rs`](examples/advanced_parquet_index.rs): Creates a detailed secondary index that covers the contents of several parquet files - [`avro_sql.rs`](examples/avro_sql.rs): Build and run a query plan from a SQL statement against a local AVRO file - [`catalog.rs`](examples/catalog.rs): Register the table into a custom catalog - [`csv_sql.rs`](examples/csv_sql.rs): Build and run a query plan from a SQL statement against a local CSV file diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs new file mode 100644 index 000000000000..9bf71e52c3de --- /dev/null +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -0,0 +1,664 @@ +// 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, Int32Array, RecordBatch, StringArray}; +use arrow_schema::SchemaRef; +use async_trait::async_trait; +use bytes::Bytes; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::parquet::{ + ParquetAccessPlan, ParquetExecBuilder, +}; +use datafusion::datasource::physical_plan::{ + parquet::ParquetFileReaderFactory, FileMeta, FileScanConfig, +}; +use datafusion::datasource::TableProvider; +use datafusion::execution::context::SessionState; +use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::parquet::arrow::arrow_reader::{ + ArrowReaderOptions, ParquetRecordBatchReaderBuilder, RowSelection, RowSelector, +}; +use datafusion::parquet::arrow::async_reader::{AsyncFileReader, ParquetObjectReader}; +use datafusion::parquet::arrow::ArrowWriter; +use datafusion::parquet::file::metadata::ParquetMetaData; +use datafusion::parquet::file::properties::{EnabledStatistics, WriterProperties}; +use datafusion::parquet::schema::types::ColumnPath; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_optimizer::pruning::PruningPredicate; +use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::*; +use datafusion_common::{ + internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, +}; +use datafusion_expr::utils::conjunction; +use datafusion_expr::{TableProviderFilterPushDown, TableType}; +use datafusion_physical_expr::utils::{Guarantee, LiteralGuarantee}; +use futures::future::BoxFuture; +use futures::FutureExt; +use object_store::ObjectStore; +use std::any::Any; +use std::collections::{HashMap, HashSet}; +use std::fs::File; +use std::ops::Range; +use std::path::{Path, PathBuf}; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; +use tempfile::TempDir; +use url::Url; + +/// This example demonstrates using low level DataFusion APIs to read only +/// certain row groups and ranges from parquet files, based on external +/// information. +/// +/// Using these APIs, you can instruct DataFusion's parquet reader to skip +/// ("prune") portions of files that do not contain relevant data. These APIs +/// can be useful for doing low latency queries over a large number of Parquet +/// files on remote storage (e.g. S3) where the cost of reading the metadata for +/// each file is high (e.g. because it requires a network round trip to the +/// storage service). +/// +/// Depending on the information from the index, DataFusion can make a request +/// to the storage service (e.g. S3) to read only the necessary data. +/// +/// Note that this example uses a hard coded index implementation. For a more +/// realistic example of creating an index to prune files, see the +/// `parquet_index.rs` example. +/// +/// Specifically, this example illustrates how to: +/// 1. Use [`ParquetFileReaderFactory`] to avoid re-reading parquet metadata on each query +/// 2. Use [`PruningPredicate`] for predicate analysis +/// 3. Pass a row group selection to [`ParuetExec`] +/// 4. Pass a row selection (within a row group) to [`ParquetExec`] +/// +/// Note this is a *VERY* low level example for people who want to build their +/// own custom indexes (e.g. for low latency queries). Most users should use +/// higher level APIs for reading parquet files: +/// [`SessionContext::read_parquet`] or [`ListingTable`], which also do file +/// pruning based on parquet statistics (using the same underlying APIs) +/// +/// # Diagram +/// +/// This diagram shows how the `ParquetExec` is configured to do only a single +/// (range) read from a parquet file, for the data that is needed. It does +/// not read the file footer or any of the row groups that are not needed. +/// +/// ```text +/// ┌───────────────────────┐ The TableProvider configures the +/// │ ┌───────────────────┐ │ ParquetExec: +/// │ │ │ │ +/// │ └───────────────────┘ │ +/// │ ┌───────────────────┐ │ +/// Row │ │ │ │ 1. To read only specific Row +/// Groups │ └───────────────────┘ │ Groups (the ParquetExec tries +/// │ ┌───────────────────┐ │ to reduce this further based +/// │ │ │ │ on metadata) +/// │ └───────────────────┘ │ ┌────────────────────┐ +/// │ ┌───────────────────┐ │ │ │ +/// │ │ │◀┼ ─ ─ ┐ │ ParquetExec │ +/// │ └───────────────────┘ │ │ (Parquet Reader) │ +/// │ ... │ └ ─ ─ ─ ─│ │ +/// │ ┌───────────────────┐ │ │ ╔═══════════════╗ │ +/// │ │ │ │ │ ║ParquetMetadata║ │ +/// │ └───────────────────┘ │ │ ╚═══════════════╝ │ +/// │ ╔═══════════════════╗ │ └────────────────────┘ +/// │ ║ Thrift metadata ║ │ +/// │ ╚═══════════════════╝ │ 1. With cached ParquetMetadata, so +/// └───────────────────────┘ the ParquetExec does not re-read / +/// Parquet File decode the thrift footer +/// +/// ``` +/// +/// Within a Row Group, Column Chunks store data in DataPages. This example also +/// shows how to configure the ParquetExec to read a `RowSelection` (row ranges) +/// which will skip unneeded data pages. This requires that the Parquet file has +/// a [Page Index]. +/// +/// ```text +/// ┌───────────────────────┐ If the RowSelection does not include any +/// │ ... │ rows from a particular Data Page, that +/// │ │ Data Page is not fetched or decoded. +/// │ ┌───────────────────┐ │ Note this requires a PageIndex +/// │ │ ┌──────────┐ │ │ +/// Row │ │ │DataPage 0│ │ │ ┌────────────────────┐ +/// Groups │ │ └──────────┘ │ │ │ │ +/// │ │ ┌──────────┐ │ │ │ ParquetExec │ +/// │ │ ... │DataPage 1│ ◀┼ ┼ ─ ─ ─ │ (Parquet Reader) │ +/// │ │ └──────────┘ │ │ └ ─ ─ ─ ─ ─│ │ +/// │ │ ┌──────────┐ │ │ │ ╔═══════════════╗ │ +/// │ │ │DataPage 2│ │ │ If only rows │ ║ParquetMetadata║ │ +/// │ │ └──────────┘ │ │ from DataPage 1 │ ╚═══════════════╝ │ +/// │ └───────────────────┘ │ are selected, └────────────────────┘ +/// │ │ only DataPage 1 +/// │ ... │ is fetched and +/// │ │ decoded +/// │ ╔═══════════════════╗ │ +/// │ ║ Thrift metadata ║ │ +/// │ ╚═══════════════════╝ │ +/// └───────────────────────┘ +/// Parquet File +/// ``` +/// +/// [`ListingTable`]: datafusion::datasource::listing::ListingTable +/// [Page Index](https://github.com/apache/parquet-format/blob/master/PageIndex.md) +#[tokio::main] +async fn main() -> Result<()> { + // the object store is used to read the parquet files (in this case, it is + // a local file system, but in a real system it could be S3, GCS, etc) + let object_store: Arc = + Arc::new(object_store::local::LocalFileSystem::new()); + + // Create a custom table provider with our special index. + let provider = Arc::new(IndexTableProvider::try_new(Arc::clone(&object_store))?); + + // SessionContext for running queries that has the table provider + // registered as "index_table" + let ctx = SessionContext::new(); + ctx.register_table("index_table", Arc::clone(&provider) as _)?; + + // register object store provider for urls like `file://` work + let url = Url::try_from("file://").unwrap(); + ctx.register_object_store(&url, object_store); + + // Select data from the table without any predicates (and thus no pruning) + println!("** Select data, no predicates:"); + ctx.sql("SELECT avg(id), max(text) FROM index_table") + .await? + .show() + .await?; + // the underlying parquet reader makes 10 IO requests, one for each row group + + // Now, run a query that has a predicate that our index can handle + // + // For this query, the access plan specifies skipping 8 row groups + // and scanning 2 of them. The skipped row groups are not read at all: + // + // [Skip, Skip, Scan, Skip, Skip, Skip, Skip, Scan, Skip, Skip] + // + // Note that the parquet reader makes 2 IO requests - one for the data from + // each row group. + println!("** Select data, predicate `id IN (250, 750)`"); + ctx.sql("SELECT text FROM index_table WHERE id IN (250, 750)") + .await? + .show() + .await?; + + // Finally, demonstrate scanning sub ranges within the row groups. + // Parquet's minimum decode unit is a page, so specifying ranges + // within a row group can be used to skip pages within a row group. + // + // For this query, the access plan specifies skipping all but the last row + // group and within the last row group, reading only the row with id 950 + // + // [Skip, Skip, Skip, Skip, Skip, Skip, Skip, Skip, Skip, Selection(skip 49, select 1, skip 50)] + // + // Note that the parquet reader makes a single IO request - for the data + // pages that must be decoded + // + // Note: in order to prune pages, the Page Index must be loaded and the + // ParquetExec will load it on demand if not present. To avoid a second IO + // during query, this example loaded the Page Index pre-emptively by setting + // `ArrowReader::with_page_index` in `IndexedFile::try_new` + provider.set_use_row_selection(true); + println!("** Select data, predicate `id = 950`"); + ctx.sql("SELECT text FROM index_table WHERE id = 950") + .await? + .show() + .await?; + + Ok(()) +} + +/// DataFusion `TableProvider` that uses knowledge of how data is distributed in +/// a file to prune row groups and rows from the file. +/// +/// `file1.parquet` contains values `0..1000` +#[derive(Debug)] +pub struct IndexTableProvider { + /// Where the file is stored (cleanup on drop) + #[allow(dead_code)] + tmpdir: TempDir, + /// The file that is being read. + indexed_file: IndexedFile, + /// The underlying object store + object_store: Arc, + /// if true, use row selections in addition to row group selections + use_row_selections: AtomicBool, +} +impl IndexTableProvider { + /// Create a new IndexTableProvider + /// * `object_store` - the object store implementation to use for reading files + pub fn try_new(object_store: Arc) -> Result { + let tmpdir = TempDir::new().expect("Can't make temporary directory"); + + let indexed_file = + IndexedFile::try_new(tmpdir.path().join("indexed_file.parquet"), 0..1000)?; + + Ok(Self { + indexed_file, + tmpdir, + object_store, + use_row_selections: AtomicBool::new(false), + }) + } + + /// set the value of use row selections + pub fn set_use_row_selection(&self, use_row_selections: bool) { + self.use_row_selections + .store(use_row_selections, Ordering::SeqCst); + } + + /// return the value of use row selections + pub fn use_row_selections(&self) -> bool { + self.use_row_selections.load(Ordering::SeqCst) + } + + /// convert filters like `a = 1`, `b = 2` + /// to a single predicate like `a = 1 AND b = 2` suitable for execution + fn filters_to_predicate( + &self, + state: &SessionState, + filters: &[Expr], + ) -> Result> { + let df_schema = DFSchema::try_from(self.schema())?; + + let predicate = conjunction(filters.to_vec()); + let predicate = predicate + .map(|predicate| state.create_physical_expr(predicate, &df_schema)) + .transpose()? + // if there are no filters, use a literal true to have a predicate + // that always evaluates to true we can pass to the index + .unwrap_or_else(|| datafusion_physical_expr::expressions::lit(true)); + + Ok(predicate) + } + + /// Returns a [`ParquetAccessPlan`] that specifies how to scan the + /// parquet file. + /// + /// A `ParquetAccessPlan` specifies which row groups and which rows within + /// those row groups to scan. + fn create_plan( + &self, + predicate: &Arc, + ) -> Result { + // In this example, we use the PruningPredicate's literal guarantees to + // analyze the predicate. In a real system, using + // `PruningPredicate::prune` would likely be easier to do. + let pruning_predicate = + PruningPredicate::try_new(Arc::clone(predicate), self.schema().clone())?; + + // The PruningPredicate's guarantees must all be satisfied in order for + // the predicate to possibly evaluate to true. + let guarantees = pruning_predicate.literal_guarantees(); + let Some(constants) = self.value_constants(guarantees) else { + return Ok(self.indexed_file.scan_all_plan()); + }; + + // Begin with a plan that skips all row groups. + let mut plan = self.indexed_file.scan_none_plan(); + + // determine which row groups have the values in the guarantees + for value in constants { + let ScalarValue::Int32(Some(val)) = value else { + // if we have unexpected type of constant, no pruning is possible + return Ok(self.indexed_file.scan_all_plan()); + }; + + // Since we know the values in the files are between 0..1000 and + // evenly distributed between in row groups, calculate in what row + // group this value appears and tell the parquet reader to read it + let val = *val as usize; + let num_rows_in_row_group = 1000 / plan.len(); + let row_group_index = val / num_rows_in_row_group; + plan.scan(row_group_index); + + // If we want to use row selections, which the parquet reader can + // use to skip data pages when the parquet file has a "page index" + // and the reader is configured to read it, add a row selection + if self.use_row_selections() { + let offset_in_row_group = val - row_group_index * num_rows_in_row_group; + let selection = RowSelection::from(vec![ + // skip rows before the desired row + RowSelector::skip(offset_in_row_group.saturating_sub(1)), + // select the actual row + RowSelector::select(1), + // skip any remaining rows in the group + RowSelector::skip(num_rows_in_row_group - offset_in_row_group), + ]); + + plan.scan_selection(row_group_index, selection); + } + } + + Ok(plan) + } + + /// Returns the set of constants that the `"id"` column must take in order + /// for the predicate to be true. + /// + /// If `None` is returned, we can't extract the necessary information from + /// the guarantees. + fn value_constants<'a>( + &self, + guarantees: &'a [LiteralGuarantee], + ) -> Option<&'a HashSet> { + // only handle a single guarantee for column in this example + if guarantees.len() != 1 { + return None; + } + let guarantee = guarantees.first()?; + + // Only handle IN guarantees for the "in" column + if guarantee.guarantee != Guarantee::In || guarantee.column.name() != "id" { + return None; + } + Some(&guarantee.literals) + } +} + +/// Stores information needed to scan a file +#[derive(Debug)] +struct IndexedFile { + /// File name + file_name: String, + /// The path of the file + path: PathBuf, + /// The size of the file + file_size: u64, + /// The pre-parsed parquet metadata for the file + metadata: Arc, + /// The arrow schema of the file + schema: SchemaRef, +} + +impl IndexedFile { + fn try_new(path: impl AsRef, value_range: Range) -> Result { + let path = path.as_ref(); + // write the actual file + make_demo_file(path, value_range)?; + + // Now, open the file and read its size and metadata + let file_name = path + .file_name() + .ok_or_else(|| internal_datafusion_err!("Invalid path"))? + .to_str() + .ok_or_else(|| internal_datafusion_err!("Invalid filename"))? + .to_string(); + let file_size = path.metadata()?.len(); + + let file = File::open(path).map_err(|e| { + DataFusionError::from(e).context(format!("Error opening file {path:?}")) + })?; + + let options = ArrowReaderOptions::new() + // Load the page index when reading metadata to cache + // so it is available to interpret row selections + .with_page_index(true); + let reader = + ParquetRecordBatchReaderBuilder::try_new_with_options(file, options)?; + let metadata = reader.metadata().clone(); + let schema = reader.schema().clone(); + + // canonicalize after writing the file + let path = std::fs::canonicalize(path)?; + + Ok(Self { + file_name, + path, + file_size, + metadata, + schema, + }) + } + + /// Return a `PartitionedFile` to scan the underlying file + /// + /// The returned value does not have any `ParquetAccessPlan` specified in + /// its extensions. + fn partitioned_file(&self) -> PartitionedFile { + PartitionedFile::new(self.path.display().to_string(), self.file_size) + } + + /// Return a `ParquetAccessPlan` that scans all row groups in the file + fn scan_all_plan(&self) -> ParquetAccessPlan { + ParquetAccessPlan::new_all(self.metadata.num_row_groups()) + } + + /// Return a `ParquetAccessPlan` that scans no row groups in the file + fn scan_none_plan(&self) -> ParquetAccessPlan { + ParquetAccessPlan::new_none(self.metadata.num_row_groups()) + } +} + +/// Implement the TableProvider trait for IndexTableProvider +/// so that we can query it as a table. +#[async_trait] +impl TableProvider for IndexTableProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.indexed_file.schema) + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + state: &SessionState, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + ) -> Result> { + let indexed_file = &self.indexed_file; + let predicate = self.filters_to_predicate(state, filters)?; + + // Figure out which row groups to scan based on the predicate + let access_plan = self.create_plan(&predicate)?; + println!("{access_plan:?}"); + + let partitioned_file = indexed_file + .partitioned_file() + // provide the starting access plan to the ParquetExec by + // storing it as "extensions" on PartitionedFile + .with_extensions(Arc::new(access_plan) as _); + + // Prepare for scanning + let schema = self.schema(); + let object_store_url = ObjectStoreUrl::parse("file://")?; + let file_scan_config = FileScanConfig::new(object_store_url, schema) + .with_limit(limit) + .with_projection(projection.cloned()) + .with_file(partitioned_file); + + // Configure a factory interface to avoid re-reading the metadata for each file + let reader_factory = + CachedParquetFileReaderFactory::new(Arc::clone(&self.object_store)) + .with_file(indexed_file); + + // Finally, put it all together into a ParquetExec + Ok(ParquetExecBuilder::new(file_scan_config) + // provide the predicate so the ParquetExec can try and prune + // row groups internally + .with_predicate(predicate) + // provide the factory to create parquet reader without re-reading metadata + .with_parquet_file_reader_factory(Arc::new(reader_factory)) + .build_arc()) + } + + /// Tell DataFusion to push filters down to the scan method + fn supports_filters_pushdown( + &self, + filters: &[&Expr], + ) -> Result> { + // Inexact because the pruning can't handle all expressions and pruning + // is not done at the row level -- there may be rows in returned files + // that do not pass the filter + Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) + } +} + +/// A custom [`ParquetFileReaderFactory`] that handles opening parquet files +/// from object storage, and uses pre-loaded metadata. + +#[derive(Debug)] +struct CachedParquetFileReaderFactory { + /// The underlying object store implementation for reading file data + object_store: Arc, + /// The parquet metadata for each file in the index, keyed by the file name + /// (e.g. `file1.parquet`) + metadata: HashMap>, +} + +impl CachedParquetFileReaderFactory { + fn new(object_store: Arc) -> Self { + Self { + object_store, + metadata: HashMap::new(), + } + } + /// Add the pre-parsed information about the file to the factor + fn with_file(mut self, indexed_file: &IndexedFile) -> Self { + self.metadata.insert( + indexed_file.file_name.clone(), + Arc::clone(&indexed_file.metadata), + ); + self + } +} + +impl ParquetFileReaderFactory for CachedParquetFileReaderFactory { + fn create_reader( + &self, + _partition_index: usize, + file_meta: FileMeta, + metadata_size_hint: Option, + _metrics: &ExecutionPlanMetricsSet, + ) -> Result> { + // for this example we ignore the partition index and metrics + // but in a real system you would likely use them to report details on + // the performance of the reader. + let filename = file_meta + .location() + .parts() + .last() + .expect("No path in location") + .as_ref() + .to_string(); + + let object_store = Arc::clone(&self.object_store); + let mut inner = ParquetObjectReader::new(object_store, file_meta.object_meta); + + if let Some(hint) = metadata_size_hint { + inner = inner.with_footer_size_hint(hint) + }; + + let metadata = self + .metadata + .get(&filename) + .expect("metadata for file not found: {filename}"); + Ok(Box::new(ParquetReaderWithCache { + filename, + metadata: Arc::clone(metadata), + inner, + })) + } +} + +/// wrapper around a ParquetObjectReader that caches metadata +struct ParquetReaderWithCache { + filename: String, + metadata: Arc, + inner: ParquetObjectReader, +} + +impl AsyncFileReader for ParquetReaderWithCache { + fn get_bytes( + &mut self, + range: Range, + ) -> BoxFuture<'_, datafusion::parquet::errors::Result> { + println!("get_bytes: {} Reading range {:?}", self.filename, range); + self.inner.get_bytes(range) + } + + fn get_byte_ranges( + &mut self, + ranges: Vec>, + ) -> BoxFuture<'_, datafusion::parquet::errors::Result>> { + println!( + "get_byte_ranges: {} Reading ranges {:?}", + self.filename, ranges + ); + self.inner.get_byte_ranges(ranges) + } + + fn get_metadata( + &mut self, + ) -> BoxFuture<'_, datafusion::parquet::errors::Result>> { + println!("get_metadata: {} returning cached metadata", self.filename); + + // return the cached metadata so the parquet reader does not read it + let metadata = self.metadata.clone(); + async move { Ok(metadata) }.boxed() + } +} + +/// Creates a new parquet file at the specified path. +/// +/// * id: Int32 +/// * text: Utf8 +/// +/// The `id` column increases sequentially from `min_value` to `max_value` +/// The `text` column is a repeating sequence of `TheTextValue{i}` +/// +/// Each row group has 100 rows +fn make_demo_file(path: impl AsRef, value_range: Range) -> Result<()> { + let path = path.as_ref(); + let file = File::create(path)?; + + let id = Int32Array::from_iter_values(value_range.clone()); + let text = + StringArray::from_iter_values(value_range.map(|i| format!("TheTextValue{i}"))); + + let batch = RecordBatch::try_from_iter(vec![ + ("id", Arc::new(id) as ArrayRef), + ("text", Arc::new(text) as ArrayRef), + ])?; + + let schema = batch.schema(); + + // enable page statistics for the tag column, + // for everything else. + let props = WriterProperties::builder() + .set_max_row_group_size(100) + // compute column chunk (per row group) statistics by default + .set_statistics_enabled(EnabledStatistics::Chunk) + // compute column page statistics for the tag column + .set_column_statistics_enabled(ColumnPath::from("tag"), EnabledStatistics::Page) + .build(); + + // write the actual values to the file + let mut writer = ArrowWriter::try_new(file, schema, Some(props))?; + writer.write(&batch)?; + writer.close()?; + + Ok(()) +} diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index 3e2bc0ad7c3a..e36a4f890644 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -127,6 +127,13 @@ impl Column { }) } + /// return the column's name. + /// + /// Note: This ignores the relation and returns the column name only. + pub fn name(&self) -> &str { + &self.name + } + /// Serialize column into a flat name string pub fn flat_name(&self) -> String { match &self.relation { diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 1c431d04cd35..c59cdba7c829 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -1393,6 +1393,13 @@ pub struct TableParquetOptions { pub key_value_metadata: HashMap>, } +impl TableParquetOptions { + /// Return new default TableParquetOptions + pub fn new() -> Self { + Self::default() + } +} + impl ConfigField for TableParquetOptions { fn visit(&self, v: &mut V, key_prefix: &str, description: &'static str) { self.global.visit(v, key_prefix, description); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs index e15e907cd9b8..ea3030664b7b 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs @@ -139,6 +139,11 @@ impl ParquetAccessPlan { self.set(idx, RowGroupAccess::Skip); } + /// scan the i-th row group + pub fn scan(&mut self, idx: usize) { + self.set(idx, RowGroupAccess::Scan); + } + /// Return true if the i-th row group should be scanned pub fn should_scan(&self, idx: usize) -> bool { self.row_groups[idx].should_scan() diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index ec21c5504c69..959e50fac81d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -186,9 +186,9 @@ pub use writer::plan_to_parquet; /// let exec = ParquetExec::builder(file_scan_config).build(); /// ``` /// -/// For a complete example, see the [`parquet_index_advanced` example]). +/// For a complete example, see the [`advanced_parquet_index` example]). /// -/// [`parquet_index_advanced` example]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/parquet_index_advanced.rs +/// [`parquet_index_advanced` example]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs /// /// # Execution Overview /// diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index e590f372253c..da8b793a5c29 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -54,7 +54,7 @@ impl RowGroupAccessPlanFilter { Self { access_plan } } - /// Return true if there are no row groups to scan + /// Return true if there are no row groups pub fn is_empty(&self) -> bool { self.access_plan.is_empty() } diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 4dd62a894518..98aff0d65898 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -471,8 +471,10 @@ pub struct PruningPredicate { /// Original physical predicate from which this predicate expr is derived /// (required for serialization) orig_expr: Arc, - /// [`LiteralGuarantee`]s that are used to try and prove a predicate can not - /// possibly evaluate to `true`. + /// [`LiteralGuarantee`]s used to try and prove a predicate can not possibly + /// evaluate to `true`. + /// + /// See [`PruningPredicate::literal_guarantees`] for more details. literal_guarantees: Vec, } @@ -595,6 +597,10 @@ impl PruningPredicate { } /// Returns a reference to the literal guarantees + /// + /// Note that **All** `LiteralGuarantee`s must be satisfied for the + /// expression to possibly be `true`. If any is not satisfied, the + /// expression is guaranteed to be `null` or `false`. pub fn literal_guarantees(&self) -> &[LiteralGuarantee] { &self.literal_guarantees }