From 014306d1749443d74d9a592089857295890dd9b1 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Thu, 31 Oct 2024 16:23:11 +0300 Subject: [PATCH 01/18] refactored nth_value --- datafusion/core/src/dataframe/mod.rs | 30 +- .../core/tests/fuzz_cases/window_fuzz.rs | 33 +- datafusion/expr/src/expr.rs | 112 +---- datafusion/expr/src/expr_schema.rs | 6 - datafusion/expr/src/lib.rs | 1 - datafusion/expr/src/window_function.rs | 26 - datafusion/functions-window/src/lib.rs | 5 + datafusion/functions-window/src/nth_value.rs | 463 ++++++++++++++++++ .../physical-expr/src/expressions/mod.rs | 1 - datafusion/physical-expr/src/window/mod.rs | 2 - .../physical-expr/src/window/nth_value.rs | 415 ---------------- .../physical-expr/src/window/window_expr.rs | 22 - .../src/windows/bounded_window_agg_exec.rs | 130 +---- datafusion/physical-plan/src/windows/mod.rs | 81 +-- datafusion/proto/src/generated/prost.rs | 84 ++-- .../proto/src/logical_plan/from_proto.rs | 19 - datafusion/proto/src/logical_plan/to_proto.rs | 6 - .../proto/src/physical_plan/from_proto.rs | 9 - .../proto/src/physical_plan/to_proto.rs | 39 +- .../tests/cases/roundtrip_physical_plan.rs | 37 +- datafusion/sql/src/expr/function.rs | 9 +- .../test_files/string/large_string.slt.temp | 65 +++ .../test_files/string/string_view.slt.temp | 43 ++ .../substrait/src/logical_plan/consumer.rs | 6 +- 24 files changed, 643 insertions(+), 1001 deletions(-) delete mode 100644 datafusion/expr/src/window_function.rs create mode 100644 datafusion/functions-window/src/nth_value.rs delete mode 100644 datafusion/physical-expr/src/window/nth_value.rs create mode 100644 datafusion/sqllogictest/test_files/string/large_string.slt.temp create mode 100644 datafusion/sqllogictest/test_files/string/string_view.slt.temp diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index e5d352a63c7a..979dc5f98a12 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1946,9 +1946,8 @@ mod tests { use datafusion_common_runtime::SpawnedTask; use datafusion_expr::expr::WindowFunction; use datafusion_expr::{ - cast, create_udf, lit, BuiltInWindowFunction, ExprFunctionExt, - ScalarFunctionImplementation, Volatility, WindowFrame, WindowFrameBound, - WindowFrameUnits, WindowFunctionDefinition, + cast, create_udf, lit, ExprFunctionExt, ScalarFunctionImplementation, Volatility, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::expr_fn::{array_agg, count_distinct}; use datafusion_functions_window::expr_fn::row_number; @@ -2172,31 +2171,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn select_with_window_exprs() -> Result<()> { - // build plan using Table API - let t = test_table().await?; - let first_row = Expr::WindowFunction(WindowFunction::new( - WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::FirstValue, - ), - vec![col("aggregate_test_100.c1")], - )) - .partition_by(vec![col("aggregate_test_100.c2")]) - .build() - .unwrap(); - let t2 = t.select(vec![col("c1"), first_row])?; - let plan = t2.plan.clone(); - - let sql_plan = create_plan( - "select c1, first_value(c1) over (partition by c2) from aggregate_test_100", - ) - .await?; - - assert_same_plan(&plan, &sql_plan); - Ok(()) - } - #[tokio::test] async fn select_with_periods() -> Result<()> { // define data with a column name that has a "." in it: diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 61b4e32ad6c9..e9945143fe50 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -34,8 +34,7 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::type_coercion::functions::data_types_with_aggregate_udf; use datafusion_expr::{ - BuiltInWindowFunction, WindowFrame, WindowFrameBound, WindowFrameUnits, - WindowFunctionDefinition, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; @@ -414,36 +413,6 @@ fn get_random_function( ), ); } - window_fn_map.insert( - "first_value", - ( - WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::FirstValue, - ), - vec![arg.clone()], - ), - ); - window_fn_map.insert( - "last_value", - ( - WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::LastValue, - ), - vec![arg.clone()], - ), - ); - window_fn_map.insert( - "nth_value", - ( - WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::NthValue, - ), - vec![ - arg.clone(), - lit(ScalarValue::Int64(Some(rng.gen_range(1..10)))), - ], - ), - ); let rand_fn_idx = rng.gen_range(0..window_fn_map.len()); let fn_name = window_fn_map.keys().collect::>()[rand_fn_idx]; diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index bda4d7ae3d7f..7e20876402ba 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -21,17 +21,13 @@ use std::collections::{HashMap, HashSet}; use std::fmt::{self, Display, Formatter, Write}; use std::hash::{Hash, Hasher}; use std::mem; -use std::str::FromStr; use std::sync::Arc; use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; use crate::utils::expr_to_columns; use crate::Volatility; -use crate::{ - udaf, BuiltInWindowFunction, ExprSchemable, Operator, Signature, WindowFrame, - WindowUDF, -}; +use crate::{udaf, ExprSchemable, Operator, Signature, WindowFrame, WindowUDF}; use arrow::datatypes::{DataType, FieldRef}; use datafusion_common::cse::HashNode; @@ -693,9 +689,6 @@ impl AggregateFunction { #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] /// Defines which implementation of an aggregate function DataFusion should call. pub enum WindowFunctionDefinition { - /// A built in aggregate function that leverages an aggregate function - /// A a built-in window function - BuiltInWindowFunction(BuiltInWindowFunction), /// A user defined aggregate function AggregateUDF(Arc), /// A user defined aggregate function @@ -711,9 +704,6 @@ impl WindowFunctionDefinition { display_name: &str, ) -> Result { match self { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => { - fun.return_type(input_expr_types) - } WindowFunctionDefinition::AggregateUDF(fun) => { fun.return_type(input_expr_types) } @@ -726,7 +716,6 @@ impl WindowFunctionDefinition { /// The signatures supported by the function `fun`. pub fn signature(&self) -> Signature { match self { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.signature(), WindowFunctionDefinition::AggregateUDF(fun) => fun.signature().clone(), WindowFunctionDefinition::WindowUDF(fun) => fun.signature().clone(), } @@ -735,7 +724,6 @@ impl WindowFunctionDefinition { /// Function's name for display pub fn name(&self) -> &str { match self { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.name(), WindowFunctionDefinition::WindowUDF(fun) => fun.name(), WindowFunctionDefinition::AggregateUDF(fun) => fun.name(), } @@ -745,19 +733,12 @@ impl WindowFunctionDefinition { impl Display for WindowFunctionDefinition { fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => Display::fmt(fun, f), WindowFunctionDefinition::AggregateUDF(fun) => Display::fmt(fun, f), WindowFunctionDefinition::WindowUDF(fun) => Display::fmt(fun, f), } } } -impl From for WindowFunctionDefinition { - fn from(value: BuiltInWindowFunction) -> Self { - Self::BuiltInWindowFunction(value) - } -} - impl From> for WindowFunctionDefinition { fn from(value: Arc) -> Self { Self::AggregateUDF(value) @@ -783,9 +764,10 @@ impl From> for WindowFunctionDefinition { /// ``` /// # use datafusion_expr::{Expr, BuiltInWindowFunction, col, ExprFunctionExt}; /// # use datafusion_expr::expr::WindowFunction; +/// use datafusion_expr::WindowFunctionDefinition::WindowUDF; /// // Create FIRST_VALUE(a) OVER (PARTITION BY b ORDER BY c) /// let expr = Expr::WindowFunction( -/// WindowFunction::new(BuiltInWindowFunction::FirstValue, vec![col("a")]) +/// WindowFunction::new(WindowUDF::, vec![col("a")]) /// ) /// .partition_by(vec![col("b")]) /// .order_by(vec![col("b").sort(true, true)]) @@ -823,23 +805,6 @@ impl WindowFunction { } } -/// Find DataFusion's built-in window function by name. -pub fn find_df_window_func(name: &str) -> Option { - let name = name.to_lowercase(); - // Code paths for window functions leveraging ordinary aggregators and - // built-in window functions are quite different, and the same function - // may have different implementations for these cases. If the sought - // function is not found among built-in window functions, we search for - // it among aggregate functions. - if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { - Some(WindowFunctionDefinition::BuiltInWindowFunction( - built_in_function, - )) - } else { - None - } -} - /// EXISTS expression #[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Exists { @@ -2525,77 +2490,6 @@ mod test { use super::*; - #[test] - fn test_first_value_return_type() -> Result<()> { - let fun = find_df_window_func("first_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8], &[true], "")?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::UInt64], &[true], "")?; - assert_eq!(DataType::UInt64, observed); - - Ok(()) - } - - #[test] - fn test_last_value_return_type() -> Result<()> { - let fun = find_df_window_func("last_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8], &[true], "")?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::Float64], &[true], "")?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_nth_value_return_type() -> Result<()> { - let fun = find_df_window_func("nth_value").unwrap(); - let observed = - fun.return_type(&[DataType::Utf8, DataType::UInt64], &[true, true], "")?; - assert_eq!(DataType::Utf8, observed); - - let observed = - fun.return_type(&[DataType::Float64, DataType::UInt64], &[true, true], "")?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_window_function_case_insensitive() -> Result<()> { - let names = vec!["first_value", "last_value", "nth_value"]; - for name in names { - let fun = find_df_window_func(name).unwrap(); - let fun2 = find_df_window_func(name.to_uppercase().as_str()).unwrap(); - assert_eq!(fun, fun2); - if fun.to_string() == "first_value" || fun.to_string() == "last_value" { - assert_eq!(fun.to_string(), name); - } else { - assert_eq!(fun.to_string(), name.to_uppercase()); - } - } - Ok(()) - } - - #[test] - fn test_find_df_window_function() { - assert_eq!( - find_df_window_func("first_value"), - Some(WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::FirstValue - )) - ); - assert_eq!( - find_df_window_func("LAST_value"), - Some(WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::LastValue - )) - ); - assert_eq!(find_df_window_func("not_exist"), None) - } - #[test] fn test_display_wildcard() { assert_eq!(format!("{}", wildcard()), "*"); diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 07a36672f272..d2c281c0077b 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -478,12 +478,6 @@ impl Expr { .map(|e| e.get_type(schema)) .collect::>>()?; match fun { - WindowFunctionDefinition::BuiltInWindowFunction(window_fun) => { - let return_type = window_fun.return_type(&data_types)?; - let nullable = - !["RANK", "NTILE", "CUME_DIST"].contains(&window_fun.name()); - Ok((return_type, nullable)) - } WindowFunctionDefinition::AggregateUDF(udaf) => { let new_types = data_types_with_aggregate_udf(&data_types, udaf) .map_err(|err| { diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 849d9604808c..d1ec7e8f7bc2 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -64,7 +64,6 @@ pub mod type_coercion; pub mod utils; pub mod var_provider; pub mod window_frame; -pub mod window_function; pub mod window_state; pub use built_in_window_function::BuiltInWindowFunction; diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs deleted file mode 100644 index be2b6575e2e9..000000000000 --- a/datafusion/expr/src/window_function.rs +++ /dev/null @@ -1,26 +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. - -use crate::{expr::WindowFunction, BuiltInWindowFunction, Expr, Literal}; - -/// Create an expression to represent the `nth_value` window function -pub fn nth_value(arg: Expr, n: i64) -> Expr { - Expr::WindowFunction(WindowFunction::new( - BuiltInWindowFunction::NthValue, - vec![arg, n.lit()], - )) -} diff --git a/datafusion/functions-window/src/lib.rs b/datafusion/functions-window/src/lib.rs index ff8542838df9..145457f32e2c 100644 --- a/datafusion/functions-window/src/lib.rs +++ b/datafusion/functions-window/src/lib.rs @@ -34,6 +34,7 @@ pub mod macros; pub mod cume_dist; pub mod lead_lag; +pub mod nth_value; pub mod ntile; pub mod rank; pub mod row_number; @@ -44,6 +45,7 @@ pub mod expr_fn { pub use super::cume_dist::cume_dist; pub use super::lead_lag::lag; pub use super::lead_lag::lead; + pub use super::nth_value::{first_value, last_value, nth_value}; pub use super::ntile::ntile; pub use super::rank::{dense_rank, percent_rank, rank}; pub use super::row_number::row_number; @@ -60,6 +62,9 @@ pub fn all_default_window_functions() -> Vec> { rank::dense_rank_udwf(), rank::percent_rank_udwf(), ntile::ntile_udwf(), + nth_value::first_value_udwf(), + nth_value::last_value_udwf(), + nth_value::nth_value_udwf(), ] } /// Registers all enabled packages with a [`FunctionRegistry`] diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs new file mode 100644 index 000000000000..e4e5e47600a1 --- /dev/null +++ b/datafusion/functions-window/src/nth_value.rs @@ -0,0 +1,463 @@ +// 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. + +//! `nth_value` window function implementation + +use std::any::Any; +use std::cmp::Ordering; +use std::fmt::Debug; +use std::ops::Range; +use std::sync::OnceLock; + +use crate::utils::{get_scalar_value_from_args, get_signed_integer}; +use datafusion_common::arrow::array::ArrayRef; +use datafusion_common::arrow::datatypes::{DataType, Field}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::window_doc_sections::DOC_SECTION_RANKING; +use datafusion_expr::window_state::WindowAggState; +use datafusion_expr::{ + Documentation, PartitionEvaluator, ReversedUDWF, Signature, TypeSignature, + Volatility, WindowUDFImpl, +}; +use datafusion_functions_window_common::field; +use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; +use field::WindowUDFFieldArgs; + +define_udwf_and_expr!( + First, + first_value, + "returns the first value in the window frame", + NthValue::first +); +define_udwf_and_expr!( + Last, + last_value, + "returns the last value in the window frame", + NthValue::last +); +define_udwf_and_expr!( + NthValue, + nth_value, + "returns the nth value in the window frame", + NthValue::nth +); + +/// Tag to differentiate special use cases of the NTH_VALUE built-in window function. +#[derive(Debug, Copy, Clone)] +pub enum NthValueKind { + First, + Last, + Nth, +} + +impl NthValueKind { + fn name(&self) -> &'static str { + match self { + NthValueKind::First => "first", + NthValueKind::Last => "last", + NthValueKind::Nth => "nth", + } + } +} + +#[derive(Debug)] +pub struct NthValue { + signature: Signature, + kind: NthValueKind, +} + +impl NthValue { + /// Create a new `nth_value` function + pub fn new(kind: NthValueKind) -> Self { + Self { + signature: Signature::one_of( + vec![ + TypeSignature::Any(0), + TypeSignature::Exact(vec![DataType::UInt64]), + ], + Volatility::Immutable, + ), + kind, + } + } + + pub fn first() -> Self { + Self::new(NthValueKind::First) + } + + pub fn last() -> Self { + Self::new(NthValueKind::Last) + } + pub fn nth() -> Self { + Self::new(NthValueKind::Nth) + } +} + +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_ntile_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_RANKING) + .with_description( + "Integer ranging from 1 to the argument value, dividing the partition as equally as possible", + ) + .build() + .unwrap() + }) +} + +impl WindowUDFImpl for NthValue { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + self.kind.name() + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn partition_evaluator( + &self, + partition_evaluator_args: PartitionEvaluatorArgs, + ) -> Result> { + let state = NthValueState { + finalized_result: None, + kind: self.kind, + }; + + let n: i64 = if matches!(self.kind, NthValueKind::Nth) { + get_scalar_value_from_args(partition_evaluator_args.input_exprs(), 1)? + .map(get_signed_integer) + .map_or(Ok(0), |v| v) + .map(|val| { + if partition_evaluator_args.is_reversed() { + -val + } else { + val + } + })? + } else { + 0 + }; + + Ok(Box::new(NthValueEvaluator { + state, + ignore_nulls: partition_evaluator_args.ignore_nulls(), + n, + })) + } + + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + let nullable = false; + + Ok(Field::new(field_args.name(), DataType::UInt64, nullable)) + } + + fn reverse_expr(&self) -> ReversedUDWF { + match self.kind { + NthValueKind::First => ReversedUDWF::Reversed(last_value_udwf()), + NthValueKind::Last => ReversedUDWF::Reversed(first_value_udwf()), + NthValueKind::Nth => ReversedUDWF::Reversed(nth_value_udwf()), + } + } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_ntile_doc()) + } +} + +#[derive(Debug, Clone)] +pub struct NthValueState { + // In certain cases, we can finalize the result early. Consider this usage: + // ``` + // FIRST_VALUE(increasing_col) OVER window AS my_first_value + // WINDOW (ORDER BY ts ASC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) AS window + // ``` + // The result will always be the first entry in the table. We can store such + // early-finalizing results and then just reuse them as necessary. This opens + // opportunities to prune our datasets. + pub finalized_result: Option, + pub kind: NthValueKind, +} + +#[derive(Debug)] +pub(crate) struct NthValueEvaluator { + state: NthValueState, + ignore_nulls: bool, + n: i64, +} + +impl PartitionEvaluator for NthValueEvaluator { + /// When the window frame has a fixed beginning (e.g UNBOUNDED PRECEDING), + /// for some functions such as FIRST_VALUE, LAST_VALUE and NTH_VALUE, we + /// can memoize the result. Once result is calculated, it will always stay + /// same. Hence, we do not need to keep past data as we process the entire + /// dataset. + fn memoize(&mut self, state: &mut WindowAggState) -> Result<()> { + let out = &state.out_col; + let size = out.len(); + let mut buffer_size = 1; + // Decide if we arrived at a final result yet: + let (is_prunable, is_reverse_direction) = match self.state.kind { + NthValueKind::First => { + let n_range = + state.window_frame_range.end - state.window_frame_range.start; + (n_range > 0 && size > 0, false) + } + NthValueKind::Last => (true, true), + NthValueKind::Nth => { + let n_range = + state.window_frame_range.end - state.window_frame_range.start; + match self.n.cmp(&0) { + Ordering::Greater => ( + n_range >= (self.n as usize) && size > (self.n as usize), + false, + ), + Ordering::Less => { + let reverse_index = (-self.n) as usize; + buffer_size = reverse_index; + // Negative index represents reverse direction. + (n_range >= reverse_index, true) + } + Ordering::Equal => (false, false), + } + } + }; + // Do not memoize results when nulls are ignored. + if is_prunable && !self.ignore_nulls { + if self.state.finalized_result.is_none() && !is_reverse_direction { + let result = ScalarValue::try_from_array(out, size - 1)?; + self.state.finalized_result = Some(result); + } + state.window_frame_range.start = + state.window_frame_range.end.saturating_sub(buffer_size); + } + Ok(()) + } + + fn evaluate( + &mut self, + values: &[ArrayRef], + range: &Range, + ) -> Result { + if let Some(ref result) = self.state.finalized_result { + Ok(result.clone()) + } else { + // FIRST_VALUE, LAST_VALUE, NTH_VALUE window functions take a single column, values will have size 1. + let arr = &values[0]; + let n_range = range.end - range.start; + if n_range == 0 { + // We produce None if the window is empty. + return ScalarValue::try_from(arr.data_type()); + } + + // Extract valid indices if ignoring nulls. + let valid_indices = if self.ignore_nulls { + // Calculate valid indices, inside the window frame boundaries + let slice = arr.slice(range.start, n_range); + let valid_indices = slice + .nulls() + .map(|nulls| { + nulls + .valid_indices() + // Add offset `range.start` to valid indices, to point correct index in the original arr. + .map(|idx| idx + range.start) + .collect::>() + }) + .unwrap_or_default(); + if valid_indices.is_empty() { + return ScalarValue::try_from(arr.data_type()); + } + Some(valid_indices) + } else { + None + }; + match self.state.kind { + NthValueKind::First => { + if let Some(valid_indices) = &valid_indices { + ScalarValue::try_from_array(arr, valid_indices[0]) + } else { + ScalarValue::try_from_array(arr, range.start) + } + } + NthValueKind::Last => { + if let Some(valid_indices) = &valid_indices { + ScalarValue::try_from_array( + arr, + valid_indices[valid_indices.len() - 1], + ) + } else { + ScalarValue::try_from_array(arr, range.end - 1) + } + } + NthValueKind::Nth => { + match self.n.cmp(&0) { + Ordering::Greater => { + // SQL indices are not 0-based. + let index = (self.n as usize) - 1; + if index >= n_range { + // Outside the range, return NULL: + ScalarValue::try_from(arr.data_type()) + } else if let Some(valid_indices) = valid_indices { + if index >= valid_indices.len() { + return ScalarValue::try_from(arr.data_type()); + } + ScalarValue::try_from_array(&arr, valid_indices[index]) + } else { + ScalarValue::try_from_array(arr, range.start + index) + } + } + Ordering::Less => { + let reverse_index = (-self.n) as usize; + if n_range < reverse_index { + // Outside the range, return NULL: + ScalarValue::try_from(arr.data_type()) + } else if let Some(valid_indices) = valid_indices { + if reverse_index > valid_indices.len() { + return ScalarValue::try_from(arr.data_type()); + } + let new_index = + valid_indices[valid_indices.len() - reverse_index]; + ScalarValue::try_from_array(&arr, new_index) + } else { + ScalarValue::try_from_array( + arr, + range.start + n_range - reverse_index, + ) + } + } + Ordering::Equal => ScalarValue::try_from(arr.data_type()), + } + } + } + } + } + + fn supports_bounded_execution(&self) -> bool { + true + } + + fn uses_window_frame(&self) -> bool { + true + } +} + +#[cfg(test)] +mod tests { + + // fn test_i32_result(expr: NthValue, expected: Int32Array) -> Result<()> { + // let arr: ArrayRef = Arc::new(Int32Array::from(vec![1, -2, 3, -4, 5, -6, 7, 8])); + // let values = vec![arr]; + // let schema = Schema::new(vec![Field::new("arr", DataType::Int32, false)]); + // let batch = RecordBatch::try_new(Arc::new(schema), values.clone())?; + // let mut ranges: Vec> = vec![]; + // for i in 0..8 { + // ranges.push(Range { + // start: 0, + // end: i + 1, + // }) + // } + // let mut evaluator = expr.create_evaluator()?; + // let values = expr.evaluate_args(&batch)?; + // let result = ranges + // .iter() + // .map(|range| evaluator.evaluate(&values, range)) + // .collect::>>()?; + // let result = ScalarValue::iter_to_array(result.into_iter())?; + // let result = as_int32_array(&result)?; + // assert_eq!(expected, *result); + // Ok(()) + // } + // + // // #[test] + // // fn first_value() -> Result<()> { + // // let first_value = NthValue::first( + // // "first_value".to_owned(), + // // Arc::new(Column::new("arr", 0)), + // // DataType::Int32, + // // false, + // // ); + // // test_i32_result(first_value, Int32Array::from(vec![1; 8]))?; + // // Ok(()) + // // } + // // + // // #[test] + // // fn last_value() -> Result<()> { + // // let last_value = NthValue::last( + // // "last_value".to_owned(), + // // Arc::new(Column::new("arr", 0)), + // // DataType::Int32, + // // false, + // // ); + // // test_i32_result( + // // last_value, + // // Int32Array::from(vec![ + // // Some(1), + // // Some(-2), + // // Some(3), + // // Some(-4), + // // Some(5), + // // Some(-6), + // // Some(7), + // // Some(8), + // // ]), + // // )?; + // // Ok(()) + // // } + // // + // // #[test] + // // fn nth_value_1() -> Result<()> { + // // let nth_value = NthValue::nth( + // // "nth_value".to_owned(), + // // Arc::new(Column::new("arr", 0)), + // // DataType::Int32, + // // 1, + // // false, + // // )?; + // // test_i32_result(nth_value, Int32Array::from(vec![1; 8]))?; + // // Ok(()) + // // } + // // + // // #[test] + // // fn nth_value_2() -> Result<()> { + // // let nth_value = NthValue::nth( + // // "nth_value".to_owned(), + // // Arc::new(Column::new("arr", 0)), + // // DataType::Int32, + // // 2, + // // false, + // // )?; + // // test_i32_result( + // // nth_value, + // // Int32Array::from(vec![ + // // None, + // // Some(-2), + // // Some(-2), + // // Some(-2), + // // Some(-2), + // // Some(-2), + // // Some(-2), + // // Some(-2), + // // ]), + // // )?; + // // Ok(()) + // // } +} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 7d71bd9ff17b..f00b49f50314 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -35,7 +35,6 @@ mod unknown_column; /// Module with some convenient methods used in expression building pub use crate::aggregate::stats::StatsType; -pub use crate::window::nth_value::NthValue; pub use crate::PhysicalSortExpr; pub use binary::{binary, similar_to, BinaryExpr}; diff --git a/datafusion/physical-expr/src/window/mod.rs b/datafusion/physical-expr/src/window/mod.rs index 3c37fff7a1ba..e7a318b860fd 100644 --- a/datafusion/physical-expr/src/window/mod.rs +++ b/datafusion/physical-expr/src/window/mod.rs @@ -18,7 +18,6 @@ mod aggregate; mod built_in; mod built_in_window_function_expr; -pub(crate) mod nth_value; mod sliding_aggregate; mod window_expr; @@ -26,7 +25,6 @@ pub use aggregate::PlainAggregateWindowExpr; pub use built_in::BuiltInWindowExpr; pub use built_in_window_function_expr::BuiltInWindowFunctionExpr; pub use sliding_aggregate::SlidingAggregateWindowExpr; -pub use window_expr::NthValueKind; pub use window_expr::PartitionBatches; pub use window_expr::PartitionKey; pub use window_expr::PartitionWindowAggStates; diff --git a/datafusion/physical-expr/src/window/nth_value.rs b/datafusion/physical-expr/src/window/nth_value.rs deleted file mode 100644 index 6ec3a23fc586..000000000000 --- a/datafusion/physical-expr/src/window/nth_value.rs +++ /dev/null @@ -1,415 +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 for `FIRST_VALUE`, `LAST_VALUE`, and `NTH_VALUE` -//! functions that can be evaluated at run time during query execution. - -use std::any::Any; -use std::cmp::Ordering; -use std::ops::Range; -use std::sync::Arc; - -use crate::window::window_expr::{NthValueKind, NthValueState}; -use crate::window::BuiltInWindowFunctionExpr; -use crate::PhysicalExpr; - -use arrow::array::{Array, ArrayRef}; -use arrow::datatypes::{DataType, Field}; -use datafusion_common::Result; -use datafusion_common::ScalarValue; -use datafusion_expr::window_state::WindowAggState; -use datafusion_expr::PartitionEvaluator; - -/// nth_value expression -#[derive(Debug)] -pub struct NthValue { - name: String, - expr: Arc, - /// Output data type - data_type: DataType, - kind: NthValueKind, - ignore_nulls: bool, -} - -impl NthValue { - /// Create a new FIRST_VALUE window aggregate function - pub fn first( - name: impl Into, - expr: Arc, - data_type: DataType, - ignore_nulls: bool, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - kind: NthValueKind::First, - ignore_nulls, - } - } - - /// Create a new LAST_VALUE window aggregate function - pub fn last( - name: impl Into, - expr: Arc, - data_type: DataType, - ignore_nulls: bool, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - kind: NthValueKind::Last, - ignore_nulls, - } - } - - /// Create a new NTH_VALUE window aggregate function - pub fn nth( - name: impl Into, - expr: Arc, - data_type: DataType, - n: i64, - ignore_nulls: bool, - ) -> Result { - Ok(Self { - name: name.into(), - expr, - data_type, - kind: NthValueKind::Nth(n), - ignore_nulls, - }) - } - - /// Get the NTH_VALUE kind - pub fn get_kind(&self) -> NthValueKind { - self.kind - } -} - -impl BuiltInWindowFunctionExpr for NthValue { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - let nullable = true; - Ok(Field::new(&self.name, self.data_type.clone(), nullable)) - } - - fn expressions(&self) -> Vec> { - vec![Arc::clone(&self.expr)] - } - - fn name(&self) -> &str { - &self.name - } - - fn create_evaluator(&self) -> Result> { - let state = NthValueState { - finalized_result: None, - kind: self.kind, - }; - Ok(Box::new(NthValueEvaluator { - state, - ignore_nulls: self.ignore_nulls, - })) - } - - fn reverse_expr(&self) -> Option> { - let reversed_kind = match self.kind { - NthValueKind::First => NthValueKind::Last, - NthValueKind::Last => NthValueKind::First, - NthValueKind::Nth(idx) => NthValueKind::Nth(-idx), - }; - Some(Arc::new(Self { - name: self.name.clone(), - expr: Arc::clone(&self.expr), - data_type: self.data_type.clone(), - kind: reversed_kind, - ignore_nulls: self.ignore_nulls, - })) - } -} - -/// Value evaluator for nth_value functions -#[derive(Debug)] -pub(crate) struct NthValueEvaluator { - state: NthValueState, - ignore_nulls: bool, -} - -impl PartitionEvaluator for NthValueEvaluator { - /// When the window frame has a fixed beginning (e.g UNBOUNDED PRECEDING), - /// for some functions such as FIRST_VALUE, LAST_VALUE and NTH_VALUE, we - /// can memoize the result. Once result is calculated, it will always stay - /// same. Hence, we do not need to keep past data as we process the entire - /// dataset. - fn memoize(&mut self, state: &mut WindowAggState) -> Result<()> { - let out = &state.out_col; - let size = out.len(); - let mut buffer_size = 1; - // Decide if we arrived at a final result yet: - let (is_prunable, is_reverse_direction) = match self.state.kind { - NthValueKind::First => { - let n_range = - state.window_frame_range.end - state.window_frame_range.start; - (n_range > 0 && size > 0, false) - } - NthValueKind::Last => (true, true), - NthValueKind::Nth(n) => { - let n_range = - state.window_frame_range.end - state.window_frame_range.start; - match n.cmp(&0) { - Ordering::Greater => { - (n_range >= (n as usize) && size > (n as usize), false) - } - Ordering::Less => { - let reverse_index = (-n) as usize; - buffer_size = reverse_index; - // Negative index represents reverse direction. - (n_range >= reverse_index, true) - } - Ordering::Equal => (false, false), - } - } - }; - // Do not memoize results when nulls are ignored. - if is_prunable && !self.ignore_nulls { - if self.state.finalized_result.is_none() && !is_reverse_direction { - let result = ScalarValue::try_from_array(out, size - 1)?; - self.state.finalized_result = Some(result); - } - state.window_frame_range.start = - state.window_frame_range.end.saturating_sub(buffer_size); - } - Ok(()) - } - - fn evaluate( - &mut self, - values: &[ArrayRef], - range: &Range, - ) -> Result { - if let Some(ref result) = self.state.finalized_result { - Ok(result.clone()) - } else { - // FIRST_VALUE, LAST_VALUE, NTH_VALUE window functions take a single column, values will have size 1. - let arr = &values[0]; - let n_range = range.end - range.start; - if n_range == 0 { - // We produce None if the window is empty. - return ScalarValue::try_from(arr.data_type()); - } - - // Extract valid indices if ignoring nulls. - let valid_indices = if self.ignore_nulls { - // Calculate valid indices, inside the window frame boundaries - let slice = arr.slice(range.start, n_range); - let valid_indices = slice - .nulls() - .map(|nulls| { - nulls - .valid_indices() - // Add offset `range.start` to valid indices, to point correct index in the original arr. - .map(|idx| idx + range.start) - .collect::>() - }) - .unwrap_or_default(); - if valid_indices.is_empty() { - return ScalarValue::try_from(arr.data_type()); - } - Some(valid_indices) - } else { - None - }; - match self.state.kind { - NthValueKind::First => { - if let Some(valid_indices) = &valid_indices { - ScalarValue::try_from_array(arr, valid_indices[0]) - } else { - ScalarValue::try_from_array(arr, range.start) - } - } - NthValueKind::Last => { - if let Some(valid_indices) = &valid_indices { - ScalarValue::try_from_array( - arr, - valid_indices[valid_indices.len() - 1], - ) - } else { - ScalarValue::try_from_array(arr, range.end - 1) - } - } - NthValueKind::Nth(n) => { - match n.cmp(&0) { - Ordering::Greater => { - // SQL indices are not 0-based. - let index = (n as usize) - 1; - if index >= n_range { - // Outside the range, return NULL: - ScalarValue::try_from(arr.data_type()) - } else if let Some(valid_indices) = valid_indices { - if index >= valid_indices.len() { - return ScalarValue::try_from(arr.data_type()); - } - ScalarValue::try_from_array(&arr, valid_indices[index]) - } else { - ScalarValue::try_from_array(arr, range.start + index) - } - } - Ordering::Less => { - let reverse_index = (-n) as usize; - if n_range < reverse_index { - // Outside the range, return NULL: - ScalarValue::try_from(arr.data_type()) - } else if let Some(valid_indices) = valid_indices { - if reverse_index > valid_indices.len() { - return ScalarValue::try_from(arr.data_type()); - } - let new_index = - valid_indices[valid_indices.len() - reverse_index]; - ScalarValue::try_from_array(&arr, new_index) - } else { - ScalarValue::try_from_array( - arr, - range.start + n_range - reverse_index, - ) - } - } - Ordering::Equal => ScalarValue::try_from(arr.data_type()), - } - } - } - } - } - - fn supports_bounded_execution(&self) -> bool { - true - } - - fn uses_window_frame(&self) -> bool { - true - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::Column; - use arrow::{array::*, datatypes::*}; - use datafusion_common::cast::as_int32_array; - - fn test_i32_result(expr: NthValue, expected: Int32Array) -> Result<()> { - let arr: ArrayRef = Arc::new(Int32Array::from(vec![1, -2, 3, -4, 5, -6, 7, 8])); - let values = vec![arr]; - let schema = Schema::new(vec![Field::new("arr", DataType::Int32, false)]); - let batch = RecordBatch::try_new(Arc::new(schema), values.clone())?; - let mut ranges: Vec> = vec![]; - for i in 0..8 { - ranges.push(Range { - start: 0, - end: i + 1, - }) - } - let mut evaluator = expr.create_evaluator()?; - let values = expr.evaluate_args(&batch)?; - let result = ranges - .iter() - .map(|range| evaluator.evaluate(&values, range)) - .collect::>>()?; - let result = ScalarValue::iter_to_array(result.into_iter())?; - let result = as_int32_array(&result)?; - assert_eq!(expected, *result); - Ok(()) - } - - #[test] - fn first_value() -> Result<()> { - let first_value = NthValue::first( - "first_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - false, - ); - test_i32_result(first_value, Int32Array::from(vec![1; 8]))?; - Ok(()) - } - - #[test] - fn last_value() -> Result<()> { - let last_value = NthValue::last( - "last_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - false, - ); - test_i32_result( - last_value, - Int32Array::from(vec![ - Some(1), - Some(-2), - Some(3), - Some(-4), - Some(5), - Some(-6), - Some(7), - Some(8), - ]), - )?; - Ok(()) - } - - #[test] - fn nth_value_1() -> Result<()> { - let nth_value = NthValue::nth( - "nth_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - 1, - false, - )?; - test_i32_result(nth_value, Int32Array::from(vec![1; 8]))?; - Ok(()) - } - - #[test] - fn nth_value_2() -> Result<()> { - let nth_value = NthValue::nth( - "nth_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - 2, - false, - )?; - test_i32_result( - nth_value, - Int32Array::from(vec![ - None, - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - ]), - )?; - Ok(()) - } -} diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 46c46fab68c5..fb3759a34f15 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -530,28 +530,6 @@ pub enum WindowFn { Aggregate(Box), } -/// Tag to differentiate special use cases of the NTH_VALUE built-in window function. -#[derive(Debug, Copy, Clone)] -pub enum NthValueKind { - First, - Last, - Nth(i64), -} - -#[derive(Debug, Clone)] -pub struct NthValueState { - // In certain cases, we can finalize the result early. Consider this usage: - // ``` - // FIRST_VALUE(increasing_col) OVER window AS my_first_value - // WINDOW (ORDER BY ts ASC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) AS window - // ``` - // The result will always be the first entry in the table. We can store such - // early-finalizing results and then just reuse them as necessary. This opens - // opportunities to prune our datasets. - pub finalized_result: Option, - pub kind: NthValueKind, -} - /// Key for IndexMap for each unique partition /// /// For instance, if window frame is `OVER(PARTITION BY a,b)`, 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 6495657339fa..9b3a18b3f1e5 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1159,8 +1159,6 @@ mod tests { use std::task::{Context, Poll}; use std::time::Duration; - use crate::common::collect; - use crate::memory::MemoryExec; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; use crate::windows::{create_window_expr, BoundedWindowAggExec, InputOrderMode}; @@ -1180,10 +1178,7 @@ mod tests { WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::expressions::{col, Column, NthValue}; - use datafusion_physical_expr::window::{ - BuiltInWindowExpr, BuiltInWindowFunctionExpr, - }; + use datafusion_physical_expr::expressions::{col, Column}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr, PhysicalSortExpr}; use futures::future::Shared; @@ -1499,129 +1494,6 @@ mod tests { Ok(source) } - // Tests NTH_VALUE(negative index) with memoize feature. - // To be able to trigger memoize feature for NTH_VALUE we need to - // - feed BoundedWindowAggExec with batch stream data. - // - Window frame should contain UNBOUNDED PRECEDING. - // It hard to ensure these conditions are met, from the sql query. - #[tokio::test] - async fn test_window_nth_value_bounded_memoize() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); - - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - // Create a new batch of data to insert into the table - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], - )?; - - let memory_exec = MemoryExec::try_new( - &[vec![batch.clone(), batch.clone(), batch.clone()]], - Arc::clone(&schema), - None, - ) - .map(|e| Arc::new(e) as Arc)?; - let col_a = col("a", &schema)?; - let nth_value_func1 = NthValue::nth( - "nth_value(-1)", - Arc::clone(&col_a), - DataType::Int32, - 1, - false, - )? - .reverse_expr() - .unwrap(); - let nth_value_func2 = NthValue::nth( - "nth_value(-2)", - Arc::clone(&col_a), - DataType::Int32, - 2, - false, - )? - .reverse_expr() - .unwrap(); - let last_value_func = Arc::new(NthValue::last( - "last", - Arc::clone(&col_a), - DataType::Int32, - false, - )) as _; - let window_exprs = vec![ - // LAST_VALUE(a) - Arc::new(BuiltInWindowExpr::new( - last_value_func, - &[], - &[], - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - // NTH_VALUE(a, -1) - Arc::new(BuiltInWindowExpr::new( - nth_value_func1, - &[], - &[], - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - // NTH_VALUE(a, -2) - Arc::new(BuiltInWindowExpr::new( - nth_value_func2, - &[], - &[], - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - ]; - let physical_plan = BoundedWindowAggExec::try_new( - window_exprs, - memory_exec, - vec![], - InputOrderMode::Sorted, - ) - .map(|e| Arc::new(e) as Arc)?; - - let batches = collect(physical_plan.execute(0, task_ctx)?).await?; - - let expected = vec![ - "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[3]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+------+---------------+---------------+", - "| a | last | nth_value(-1) | nth_value(-2) |", - "+---+------+---------------+---------------+", - "| 1 | 1 | 1 | |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "| 1 | 1 | 1 | 3 |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "| 1 | 1 | 1 | 3 |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "+---+------+---------------+---------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - // This test, tests whether most recent row guarantee by the input batch of the `BoundedWindowAggExec` // helps `BoundedWindowAggExec` to generate low latency result in the `Linear` mode. // Input data generated at the source is diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 7ebb7e71ec57..7602ec3d6211 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -21,16 +21,15 @@ use std::borrow::Borrow; use std::sync::Arc; use crate::{ - expressions::{Literal, NthValue, PhysicalSortExpr}, - ExecutionPlan, ExecutionPlanProperties, InputOrderMode, PhysicalExpr, + expressions::PhysicalSortExpr, ExecutionPlan, ExecutionPlanProperties, + InputOrderMode, PhysicalExpr, }; use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::{exec_datafusion_err, exec_err, Result, ScalarValue}; +use datafusion_common::{exec_err, Result}; use datafusion_expr::{ - BuiltInWindowFunction, PartitionEvaluator, ReversedUDWF, WindowFrame, - WindowFunctionDefinition, WindowUDF, + PartitionEvaluator, ReversedUDWF, WindowFrame, WindowFunctionDefinition, WindowUDF, }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::equivalence::collapse_lex_req; @@ -104,14 +103,6 @@ pub fn create_window_expr( ignore_nulls: bool, ) -> Result> { Ok(match fun { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => { - Arc::new(BuiltInWindowExpr::new( - create_built_in_window_expr(fun, args, input_schema, name, ignore_nulls)?, - partition_by, - order_by, - window_frame, - )) - } WindowFunctionDefinition::AggregateUDF(fun) => { let aggregate = AggregateExprBuilder::new(Arc::clone(fun), args.to_vec()) .schema(Arc::new(input_schema.clone())) @@ -163,70 +154,6 @@ fn window_expr_from_aggregate_expr( } } -fn get_signed_integer(value: ScalarValue) -> Result { - if value.is_null() { - return Ok(0); - } - - if !value.data_type().is_integer() { - return exec_err!("Expected an integer value"); - } - - value.cast_to(&DataType::Int64)?.try_into() -} - -fn create_built_in_window_expr( - fun: &BuiltInWindowFunction, - args: &[Arc], - input_schema: &Schema, - name: String, - ignore_nulls: bool, -) -> Result> { - // derive the output datatype from incoming schema - let out_data_type: &DataType = input_schema.field_with_name(&name)?.data_type(); - - Ok(match fun { - BuiltInWindowFunction::NthValue => { - let arg = Arc::clone(&args[0]); - let n = get_signed_integer( - args[1] - .as_any() - .downcast_ref::() - .ok_or_else(|| { - exec_datafusion_err!("Expected a signed integer literal for the second argument of nth_value, got {}", args[1]) - })? - .value() - .clone(), - )?; - Arc::new(NthValue::nth( - name, - arg, - out_data_type.clone(), - n, - ignore_nulls, - )?) - } - BuiltInWindowFunction::FirstValue => { - let arg = Arc::clone(&args[0]); - Arc::new(NthValue::first( - name, - arg, - out_data_type.clone(), - ignore_nulls, - )) - } - BuiltInWindowFunction::LastValue => { - let arg = Arc::clone(&args[0]); - Arc::new(NthValue::last( - name, - arg, - out_data_type.clone(), - ignore_nulls, - )) - } - }) -} - /// Creates a `BuiltInWindowFunctionExpr` suitable for a user defined window function fn create_udwf_window_expr( fun: &Arc, diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index dfc30e809108..228f20d98b9a 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -117,10 +117,11 @@ pub struct ListingTableScanNode { pub target_partitions: u32, #[prost(message, repeated, tag = "13")] pub file_sort_order: ::prost::alloc::vec::Vec, - #[prost(oneof = "listing_table_scan_node::FileFormatType", tags = "10, 11, 12, 15")] - pub file_format_type: ::core::option::Option< - listing_table_scan_node::FileFormatType, - >, + #[prost( + oneof = "listing_table_scan_node::FileFormatType", + tags = "10, 11, 12, 15" + )] + pub file_format_type: ::core::option::Option, } /// Nested message and enum types in `ListingTableScanNode`. pub mod listing_table_scan_node { @@ -256,10 +257,8 @@ pub struct CreateExternalTableNode { #[prost(message, optional, tag = "12")] pub constraints: ::core::option::Option, #[prost(map = "string, message", tag = "13")] - pub column_defaults: ::std::collections::HashMap< - ::prost::alloc::string::String, - LogicalExprNode, - >, + pub column_defaults: + ::std::collections::HashMap<::prost::alloc::string::String, LogicalExprNode>, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct PrepareNode { @@ -745,8 +744,6 @@ pub struct WindowExprNode { pub mod window_expr_node { #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum WindowFunction { - #[prost(enumeration = "super::BuiltInWindowFunction", tag = "2")] - BuiltInFunction(i32), #[prost(string, tag = "3")] Udaf(::prost::alloc::string::String), #[prost(string, tag = "9")] @@ -953,9 +950,7 @@ pub struct FullTableReference { #[derive(Clone, PartialEq, ::prost::Message)] pub struct TableReference { #[prost(oneof = "table_reference::TableReferenceEnum", tags = "1, 2, 3")] - pub table_reference_enum: ::core::option::Option< - table_reference::TableReferenceEnum, - >, + pub table_reference_enum: ::core::option::Option, } /// Nested message and enum types in `TableReference`. pub mod table_reference { @@ -1073,9 +1068,8 @@ pub struct JsonSink { #[prost(message, optional, tag = "1")] pub config: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub writer_options: ::core::option::Option< - super::datafusion_common::JsonWriterOptions, - >, + pub writer_options: + ::core::option::Option, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct JsonSinkExecNode { @@ -1093,9 +1087,8 @@ pub struct CsvSink { #[prost(message, optional, tag = "1")] pub config: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub writer_options: ::core::option::Option< - super::datafusion_common::CsvWriterOptions, - >, + pub writer_options: + ::core::option::Option, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct CsvSinkExecNode { @@ -1113,9 +1106,8 @@ pub struct ParquetSink { #[prost(message, optional, tag = "1")] pub config: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub parquet_options: ::core::option::Option< - super::datafusion_common::TableParquetOptions, - >, + pub parquet_options: + ::core::option::Option, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetSinkExecNode { @@ -1235,9 +1227,8 @@ pub struct PhysicalAggregateExprNode { #[prost(bytes = "vec", optional, tag = "7")] pub fun_definition: ::core::option::Option<::prost::alloc::vec::Vec>, #[prost(oneof = "physical_aggregate_expr_node::AggregateFunction", tags = "4")] - pub aggregate_function: ::core::option::Option< - physical_aggregate_expr_node::AggregateFunction, - >, + pub aggregate_function: + ::core::option::Option, } /// Nested message and enum types in `PhysicalAggregateExprNode`. pub mod physical_aggregate_expr_node { @@ -1262,16 +1253,13 @@ pub struct PhysicalWindowExprNode { #[prost(bytes = "vec", optional, tag = "9")] pub fun_definition: ::core::option::Option<::prost::alloc::vec::Vec>, #[prost(oneof = "physical_window_expr_node::WindowFunction", tags = "2, 3")] - pub window_function: ::core::option::Option< - physical_window_expr_node::WindowFunction, - >, + pub window_function: + ::core::option::Option, } /// Nested message and enum types in `PhysicalWindowExprNode`. pub mod physical_window_expr_node { #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum WindowFunction { - #[prost(enumeration = "super::BuiltInWindowFunction", tag = "2")] - BuiltInFunction(i32), #[prost(string, tag = "3")] UserDefinedAggrFunction(::prost::alloc::string::String), } @@ -1783,9 +1771,7 @@ pub struct PartitionedFile { #[prost(uint64, tag = "3")] pub last_modified_ns: u64, #[prost(message, repeated, tag = "4")] - pub partition_values: ::prost::alloc::vec::Vec< - super::datafusion_common::ScalarValue, - >, + pub partition_values: ::prost::alloc::vec::Vec, #[prost(message, optional, tag = "5")] pub range: ::core::option::Option, #[prost(message, optional, tag = "6")] @@ -1809,7 +1795,9 @@ pub struct PartitionStats { #[prost(message, repeated, tag = "4")] pub column_stats: ::prost::alloc::vec::Vec, } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum BuiltInWindowFunction { /// @@ -1850,7 +1838,9 @@ impl BuiltInWindowFunction { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum WindowFrameUnits { Rows = 0, @@ -1879,7 +1869,9 @@ impl WindowFrameUnits { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum WindowFrameBoundType { CurrentRow = 0, @@ -1908,7 +1900,9 @@ impl WindowFrameBoundType { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum DateUnit { Day = 0, @@ -1934,7 +1928,9 @@ impl DateUnit { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum InsertOp { Append = 0, @@ -1963,7 +1959,9 @@ impl InsertOp { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum PartitionMode { CollectLeft = 0, @@ -1992,7 +1990,9 @@ impl PartitionMode { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum StreamPartitionMode { SinglePartition = 0, @@ -2018,7 +2018,9 @@ impl StreamPartitionMode { } } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[derive( + Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, +)] #[repr(i32)] pub enum AggregateMode { Partial = 0, diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 27bda7dd5ace..2f151b0aed11 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -286,25 +286,6 @@ pub fn parse_expr( // TODO: support proto for null treatment match window_function { - window_expr_node::WindowFunction::BuiltInFunction(i) => { - let built_in_function = protobuf::BuiltInWindowFunction::try_from(*i) - .map_err(|_| Error::unknown("BuiltInWindowFunction", *i))? - .into(); - - let args = parse_exprs(&expr.exprs, registry, codec)?; - - Expr::WindowFunction(WindowFunction::new( - expr::WindowFunctionDefinition::BuiltInWindowFunction( - built_in_function, - ), - args, - )) - .partition_by(partition_by) - .order_by(order_by) - .window_frame(window_frame) - .build() - .map_err(Error::DataFusionError) - } window_expr_node::WindowFunction::Udaf(udaf_name) => { let udaf_function = match &expr.fun_definition { Some(buf) => codec.try_decode_udaf(udaf_name, buf)?, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 5a6f3a32c668..fa94f25e74cc 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -312,12 +312,6 @@ pub fn serialize_expr( null_treatment: _, }) => { let (window_function, fun_definition) = match fun { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => ( - protobuf::window_expr_node::WindowFunction::BuiltInFunction( - protobuf::BuiltInWindowFunction::from(fun).into(), - ), - None, - ), WindowFunctionDefinition::AggregateUDF(aggr_udf) => { let mut buf = Vec::new(); let _ = codec.try_encode_udaf(aggr_udf, &mut buf); diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 20ec5eeaeaf8..6f77409dcc6b 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -146,15 +146,6 @@ pub fn parse_physical_window_expr( let fun = if let Some(window_func) = proto.window_function.as_ref() { match window_func { - protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(n) => { - let f = protobuf::BuiltInWindowFunction::try_from(*n).map_err(|_| { - proto_error(format!( - "Received an unknown window builtin function: {n}" - )) - })?; - - WindowFunctionDefinition::BuiltInWindowFunction(f.into()) - } protobuf::physical_window_expr_node::WindowFunction::UserDefinedAggrFunction(udaf_name) => { WindowFunctionDefinition::AggregateUDF(match &proto.fun_definition { Some(buf) => codec.try_decode_udaf(udaf_name, buf)?, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 89a2403922e9..4a112d749124 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -20,14 +20,14 @@ use std::sync::Arc; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; -use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; +use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, - Literal, NegativeExpr, NotExpr, NthValue, TryCastExpr, + Literal, NegativeExpr, NotExpr, TryCastExpr, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; -use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; +use datafusion::physical_plan::windows::PlainAggregateWindowExpr; use datafusion::physical_plan::{Partitioning, PhysicalExpr, WindowExpr}; use datafusion::{ datasource::{ @@ -99,39 +99,10 @@ pub fn serialize_physical_window_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let expr = window_expr.as_any(); - let mut args = window_expr.expressions().to_vec(); + let args = window_expr.expressions().to_vec(); let window_frame = window_expr.get_window_frame(); - let (window_function, fun_definition) = if let Some(built_in_window_expr) = - expr.downcast_ref::() - { - let expr = built_in_window_expr.get_built_in_func_expr(); - let built_in_fn_expr = expr.as_any(); - - let builtin_fn = - if let Some(nth_value_expr) = built_in_fn_expr.downcast_ref::() { - match nth_value_expr.get_kind() { - NthValueKind::First => protobuf::BuiltInWindowFunction::FirstValue, - NthValueKind::Last => protobuf::BuiltInWindowFunction::LastValue, - NthValueKind::Nth(n) => { - args.insert( - 1, - Arc::new(Literal::new( - datafusion_common::ScalarValue::Int64(Some(n)), - )), - ); - protobuf::BuiltInWindowFunction::NthValue - } - } - } else { - return not_impl_err!("BuiltIn function not supported: {expr:?}"); - }; - - ( - physical_window_expr_node::WindowFunction::BuiltInFunction(builtin_fn as i32), - None, - ) - } else if let Some(plain_aggr_window_expr) = + let (window_function, fun_definition) = if let Some(plain_aggr_window_expr) = expr.downcast_ref::() { serialize_physical_window_aggr_expr( diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 4a9bf6afb49e..695216175987 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -60,8 +60,7 @@ use datafusion::physical_plan::aggregates::{ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ - binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, NthValue, - PhysicalSortExpr, + binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, PhysicalSortExpr, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::DataSinkExec; @@ -75,9 +74,7 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; -use datafusion::physical_plan::windows::{ - BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, -}; +use datafusion::physical_plan::windows::{PlainAggregateWindowExpr, WindowAggExec}; use datafusion::physical_plan::{ExecutionPlan, Partitioning, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; @@ -273,30 +270,6 @@ fn roundtrip_window() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let window_frame = WindowFrame::new_bounds( - datafusion_expr::WindowFrameUnits::Range, - WindowFrameBound::Preceding(ScalarValue::Int64(None)), - WindowFrameBound::CurrentRow, - ); - - let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( - Arc::new(NthValue::first( - "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", - col("a", &schema)?, - DataType::Int64, - false, - )), - &[col("b", &schema)?], - &[PhysicalSortExpr { - expr: col("a", &schema)?, - options: SortOptions { - descending: false, - nulls_first: false, - }, - }], - Arc::new(window_frame), - )); - let plain_aggr_window_expr = Arc::new(PlainAggregateWindowExpr::new( AggregateExprBuilder::new( avg_udaf(), @@ -334,11 +307,7 @@ fn roundtrip_window() -> Result<()> { let input = Arc::new(EmptyExec::new(schema.clone())); roundtrip_test(Arc::new(WindowAggExec::try_new( - vec![ - builtin_window_expr, - plain_aggr_window_expr, - sliding_aggr_window_expr, - ], + vec![plain_aggr_window_expr, sliding_aggr_window_expr], input, vec![col("b", &schema)?], )?)) diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 619eadcf0fb8..a569119f7d08 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -393,12 +393,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) { Ok(WindowFunctionDefinition::AggregateUDF(udaf.unwrap())) } else { - expr::find_df_window_func(name) - .or_else(|| { - self.context_provider - .get_window_meta(name) - .map(WindowFunctionDefinition::WindowUDF) - }) + self.context_provider + .get_window_meta(name) + .map(WindowFunctionDefinition::WindowUDF) .ok_or_else(|| { plan_datafusion_err!("There is no window function named {name}") }) diff --git a/datafusion/sqllogictest/test_files/string/large_string.slt.temp b/datafusion/sqllogictest/test_files/string/large_string.slt.temp new file mode 100644 index 000000000000..fa65714c6628 --- /dev/null +++ b/datafusion/sqllogictest/test_files/string/large_string.slt.temp @@ -0,0 +1,65 @@ +# 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. + +include ./init_data.slt.part + +# -------------------------------------- +# Setup test tables with different physical string types +# and repeat tests in `string_query.slt.part` +# -------------------------------------- +statement ok +create table test_basic_operator as +select + arrow_cast(column1, 'LargeUtf8') as ascii_1, + arrow_cast(column2, 'LargeUtf8') as ascii_2, + arrow_cast(column3, 'LargeUtf8') as unicode_1, + arrow_cast(column4, 'LargeUtf8') as unicode_2 +from test_source; + +statement ok +create table test_substr as +select arrow_cast(col1, 'LargeUtf8') as c1 from test_substr_base; + +# select +query TTTT +SELECT ascii_1, ascii_2, unicode_1, unicode_2 FROM test_basic_operator +---- +Andrew X datafusion📊🔥 🔥 +Xiangpeng Xiangpeng datafusion数据融合 datafusion数据融合 +Raphael R datafusionДатаФусион аФус +NULL R NULL 🔥 + +# TODO: move it back to `string_query.slt.part` after fixing the issue +# see detail: https://github.com/apache/datafusion/issues/12637 +# Test pattern with wildcard characters +query TTBBBB +select ascii_1, unicode_1, + ascii_1 like 'An%' as ascii_like, + unicode_1 like '%ion数据%' as unicode_like, + ascii_1 ilike 'An%' as ascii_ilike, + unicode_1 ilike '%ion数据%' as unicode_ilik +from test_basic_operator; +---- +Andrew datafusion📊🔥 true false true false +Xiangpeng datafusion数据融合 false true false true +Raphael datafusionДатаФусион false false false false +NULL NULL NULL NULL NULL NULL + +# +# common test for string-like functions and operators +# +include ./string_query.slt.part diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt.temp b/datafusion/sqllogictest/test_files/string/string_view.slt.temp new file mode 100644 index 000000000000..183ce75e0667 --- /dev/null +++ b/datafusion/sqllogictest/test_files/string/string_view.slt.temp @@ -0,0 +1,43 @@ +# 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. + +include ./init_data.slt.part + +# -------------------------------------- +# Setup test tables with different physical string types +# and repeat tests in `string_query.slt.part` +# -------------------------------------- +statement ok +create table test_basic_operator as +select + arrow_cast(column1, 'Utf8View') as ascii_1, + arrow_cast(column2, 'Utf8View') as ascii_2, + arrow_cast(column3, 'Utf8View') as unicode_1, + arrow_cast(column4, 'Utf8View') as unicode_2 +from test_source; + +statement ok +create table test_substr as +select arrow_cast(col1, 'Utf8View') as c1 from test_substr_base; + +statement ok +drop table test_source + +# +# common test for string-like functions and operators +# +include ./string_query.slt.part diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 2aaf8ec0aa06..f3e5ac4d143f 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -30,8 +30,8 @@ use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::expr::{Exists, InSubquery, Sort}; use datafusion::logical_expr::{ - expr::find_df_window_func, Aggregate, BinaryExpr, Case, EmptyRelation, Expr, - ExprSchemable, LogicalPlan, Operator, Projection, SortExpr, Values, + Aggregate, BinaryExpr, Case, EmptyRelation, Expr, ExprSchemable, LogicalPlan, + Operator, Projection, SortExpr, Values, }; use substrait::proto::expression::subquery::set_predicate::PredicateOp; use substrait::proto::expression_reference::ExprType; @@ -1529,8 +1529,6 @@ pub async fn from_substrait_rex( Ok(WindowFunctionDefinition::WindowUDF(udwf)) } else if let Ok(udaf) = ctx.udaf(fn_name) { Ok(WindowFunctionDefinition::AggregateUDF(udaf)) - } else if let Some(fun) = find_df_window_func(fn_name) { - Ok(fun) } else { not_impl_err!( "Window function {} is not supported: function anchor = {:?}", From fda6a6f21d19cadc8460664a446acabb165711b2 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Sun, 3 Nov 2024 23:17:54 +0300 Subject: [PATCH 02/18] continue --- .../expr/src/built_in_window_function.rs | 105 +------ datafusion/functions-window/src/lib.rs | 3 + datafusion/functions-window/src/nth_value.rs | 290 +++++++++++------- datafusion/proto/src/generated/prost.rs | 84 +++-- datafusion/sql/src/expr/function.rs | 7 +- .../test_files/string/large_string.slt.temp | 65 ---- .../test_files/string/string_view.slt.temp | 43 --- 7 files changed, 220 insertions(+), 377 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/string/large_string.slt.temp delete mode 100644 datafusion/sqllogictest/test_files/string/string_view.slt.temp diff --git a/datafusion/expr/src/built_in_window_function.rs b/datafusion/expr/src/built_in_window_function.rs index ab41395ad371..b1ba6d239ada 100644 --- a/datafusion/expr/src/built_in_window_function.rs +++ b/datafusion/expr/src/built_in_window_function.rs @@ -17,115 +17,12 @@ //! Built-in functions module contains all the built-in functions definitions. -use std::fmt; -use std::str::FromStr; - -use crate::type_coercion::functions::data_types; -use crate::utils; -use crate::{Signature, Volatility}; -use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; - -use arrow::datatypes::DataType; - use strum_macros::EnumIter; -impl fmt::Display for BuiltInWindowFunction { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", self.name()) - } -} - /// A [window function] built in to DataFusion /// /// [Window Function]: https://en.wikipedia.org/wiki/Window_function_(SQL) #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, EnumIter)] pub enum BuiltInWindowFunction { - /// returns value evaluated at the row that is the first row of the window frame - FirstValue, - /// Returns value evaluated at the row that is the last row of the window frame - LastValue, - /// Returns value evaluated at the row that is the nth row of the window frame (counting from 1); returns null if no such row - NthValue, -} - -impl BuiltInWindowFunction { - pub fn name(&self) -> &str { - use BuiltInWindowFunction::*; - match self { - FirstValue => "first_value", - LastValue => "last_value", - NthValue => "NTH_VALUE", - } - } -} - -impl FromStr for BuiltInWindowFunction { - type Err = DataFusionError; - fn from_str(name: &str) -> Result { - Ok(match name.to_uppercase().as_str() { - "FIRST_VALUE" => BuiltInWindowFunction::FirstValue, - "LAST_VALUE" => BuiltInWindowFunction::LastValue, - "NTH_VALUE" => BuiltInWindowFunction::NthValue, - _ => return plan_err!("There is no built-in window function named {name}"), - }) - } -} - -/// Returns the datatype of the built-in window function -impl BuiltInWindowFunction { - pub fn return_type(&self, input_expr_types: &[DataType]) -> Result { - // Note that this function *must* return the same type that the respective physical expression returns - // or the execution panics. - - // Verify that this is a valid set of data types for this function - data_types(input_expr_types, &self.signature()) - // Original errors are all related to wrong function signature - // Aggregate them for better error message - .map_err(|_| { - plan_datafusion_err!( - "{}", - utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types, - ) - ) - })?; - - match self { - BuiltInWindowFunction::FirstValue - | BuiltInWindowFunction::LastValue - | BuiltInWindowFunction::NthValue => Ok(input_expr_types[0].clone()), - } - } - - /// The signatures supported by the built-in window function `fun`. - pub fn signature(&self) -> Signature { - // Note: The physical expression must accept the type returned by this function or the execution panics. - match self { - BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue => { - Signature::any(1, Volatility::Immutable) - } - BuiltInWindowFunction::NthValue => Signature::any(2, Volatility::Immutable), - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use strum::IntoEnumIterator; - #[test] - // Test for BuiltInWindowFunction's Display and from_str() implementations. - // For each variant in BuiltInWindowFunction, it converts the variant to a string - // and then back to a variant. The test asserts that the original variant and - // the reconstructed variant are the same. This assertion is also necessary for - // function suggestion. See https://github.com/apache/datafusion/issues/8082 - fn test_display_and_from_str() { - for func_original in BuiltInWindowFunction::iter() { - let func_name = func_original.to_string(); - let func_from_str = BuiltInWindowFunction::from_str(&func_name).unwrap(); - assert_eq!(func_from_str, func_original); - } - } + Stub, } diff --git a/datafusion/functions-window/src/lib.rs b/datafusion/functions-window/src/lib.rs index 145457f32e2c..de6e25bd454f 100644 --- a/datafusion/functions-window/src/lib.rs +++ b/datafusion/functions-window/src/lib.rs @@ -22,6 +22,9 @@ //! //! [DataFusion]: https://crates.io/crates/datafusion //! + +extern crate core; + use std::sync::Arc; use log::debug; diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index e4e5e47600a1..5e36c15ce91d 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -17,17 +17,18 @@ //! `nth_value` window function implementation +use crate::utils::{get_scalar_value_from_args, get_signed_integer}; + use std::any::Any; use std::cmp::Ordering; use std::fmt::Debug; use std::ops::Range; use std::sync::OnceLock; -use crate::utils::{get_scalar_value_from_args, get_signed_integer}; use datafusion_common::arrow::array::ArrayRef; use datafusion_common::arrow::datatypes::{DataType, Field}; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::window_doc_sections::DOC_SECTION_RANKING; +use datafusion_expr::window_doc_sections::DOC_SECTION_ANALYTICAL; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::{ Documentation, PartitionEvaluator, ReversedUDWF, Signature, TypeSignature, @@ -35,25 +36,26 @@ use datafusion_expr::{ }; use datafusion_functions_window_common::field; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; + use field::WindowUDFFieldArgs; define_udwf_and_expr!( First, first_value, "returns the first value in the window frame", - NthValue::first + NthValue::first_value ); define_udwf_and_expr!( Last, last_value, "returns the last value in the window frame", - NthValue::last + NthValue::last_value ); define_udwf_and_expr!( NthValue, nth_value, "returns the nth value in the window frame", - NthValue::nth + NthValue::nth_value ); /// Tag to differentiate special use cases of the NTH_VALUE built-in window function. @@ -67,9 +69,9 @@ pub enum NthValueKind { impl NthValueKind { fn name(&self) -> &'static str { match self { - NthValueKind::First => "first", - NthValueKind::Last => "last", - NthValueKind::Nth => "nth", + NthValueKind::First => "first_value", + NthValueKind::Last => "last_value", + NthValueKind::Nth => "nth_value", } } } @@ -87,7 +89,8 @@ impl NthValue { signature: Signature::one_of( vec![ TypeSignature::Any(0), - TypeSignature::Exact(vec![DataType::UInt64]), + TypeSignature::Any(1), + TypeSignature::Any(2), ], Volatility::Immutable, ), @@ -95,27 +98,69 @@ impl NthValue { } } - pub fn first() -> Self { + pub fn first_value() -> Self { Self::new(NthValueKind::First) } - pub fn last() -> Self { + pub fn last_value() -> Self { Self::new(NthValueKind::Last) } - pub fn nth() -> Self { + pub fn nth_value() -> Self { Self::new(NthValueKind::Nth) } } -static DOCUMENTATION: OnceLock = OnceLock::new(); +static FIRST_VALUE_DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_first_value_doc() -> &'static Documentation { + FIRST_VALUE_DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_ANALYTICAL) + .with_description( + "Returns value evaluated at the row that is the first row of the window \ + frame.", + ) + .with_syntax_example("first_value(expression)") + .with_argument("expression", "Expression to operate on") + .build() + .unwrap() + }) +} + +static LAST_VALUE_DOCUMENTATION: OnceLock = OnceLock::new(); -fn get_ntile_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { +fn get_last_value_doc() -> &'static Documentation { + LAST_VALUE_DOCUMENTATION.get_or_init(|| { Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) + .with_doc_section(DOC_SECTION_ANALYTICAL) .with_description( - "Integer ranging from 1 to the argument value, dividing the partition as equally as possible", + "Returns value evaluated at the row that is the last row of the window \ + frame.", ) + .with_syntax_example("last_value(expression)") + .with_argument("expression", "Expression to operate on") + .build() + .unwrap() + }) +} + +static NTH_VALUE_DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_nth_value_doc() -> &'static Documentation { + NTH_VALUE_DOCUMENTATION.get_or_init(|| { + Documentation::builder() + .with_doc_section(DOC_SECTION_ANALYTICAL) + .with_description( + "Returns value evaluated at the row that is the nth row of the window \ + frame (counting from 1); null if no such row.", + ) + .with_syntax_example("nth_value(expression, n)") + .with_argument( + "expression", + "The name the column of which nth \ + value to retrieve", + ) + .with_argument("n", "Integer. Specifies the n in nth") .build() .unwrap() }) @@ -166,7 +211,7 @@ impl WindowUDFImpl for NthValue { } fn field(&self, field_args: WindowUDFFieldArgs) -> Result { - let nullable = false; + let nullable = true; Ok(Field::new(field_args.name(), DataType::UInt64, nullable)) } @@ -180,7 +225,11 @@ impl WindowUDFImpl for NthValue { } fn documentation(&self) -> Option<&Documentation> { - Some(get_ntile_doc()) + match self.kind { + NthValueKind::First => Some(get_first_value_doc()), + NthValueKind::Last => Some(get_last_value_doc()), + NthValueKind::Nth => Some(get_nth_value_doc()), + } } } @@ -362,102 +411,111 @@ impl PartitionEvaluator for NthValueEvaluator { #[cfg(test)] mod tests { + use super::*; + use arrow::array::*; + use datafusion_common::cast::as_int32_array; + use datafusion_physical_expr::expressions::{Column, Literal}; + use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use std::sync::Arc; - // fn test_i32_result(expr: NthValue, expected: Int32Array) -> Result<()> { - // let arr: ArrayRef = Arc::new(Int32Array::from(vec![1, -2, 3, -4, 5, -6, 7, 8])); - // let values = vec![arr]; - // let schema = Schema::new(vec![Field::new("arr", DataType::Int32, false)]); - // let batch = RecordBatch::try_new(Arc::new(schema), values.clone())?; - // let mut ranges: Vec> = vec![]; - // for i in 0..8 { - // ranges.push(Range { - // start: 0, - // end: i + 1, - // }) - // } - // let mut evaluator = expr.create_evaluator()?; - // let values = expr.evaluate_args(&batch)?; - // let result = ranges - // .iter() - // .map(|range| evaluator.evaluate(&values, range)) - // .collect::>>()?; - // let result = ScalarValue::iter_to_array(result.into_iter())?; - // let result = as_int32_array(&result)?; - // assert_eq!(expected, *result); - // Ok(()) - // } - // - // // #[test] - // // fn first_value() -> Result<()> { - // // let first_value = NthValue::first( - // // "first_value".to_owned(), - // // Arc::new(Column::new("arr", 0)), - // // DataType::Int32, - // // false, - // // ); - // // test_i32_result(first_value, Int32Array::from(vec![1; 8]))?; - // // Ok(()) - // // } - // // - // // #[test] - // // fn last_value() -> Result<()> { - // // let last_value = NthValue::last( - // // "last_value".to_owned(), - // // Arc::new(Column::new("arr", 0)), - // // DataType::Int32, - // // false, - // // ); - // // test_i32_result( - // // last_value, - // // Int32Array::from(vec![ - // // Some(1), - // // Some(-2), - // // Some(3), - // // Some(-4), - // // Some(5), - // // Some(-6), - // // Some(7), - // // Some(8), - // // ]), - // // )?; - // // Ok(()) - // // } - // // - // // #[test] - // // fn nth_value_1() -> Result<()> { - // // let nth_value = NthValue::nth( - // // "nth_value".to_owned(), - // // Arc::new(Column::new("arr", 0)), - // // DataType::Int32, - // // 1, - // // false, - // // )?; - // // test_i32_result(nth_value, Int32Array::from(vec![1; 8]))?; - // // Ok(()) - // // } - // // - // // #[test] - // // fn nth_value_2() -> Result<()> { - // // let nth_value = NthValue::nth( - // // "nth_value".to_owned(), - // // Arc::new(Column::new("arr", 0)), - // // DataType::Int32, - // // 2, - // // false, - // // )?; - // // test_i32_result( - // // nth_value, - // // Int32Array::from(vec![ - // // None, - // // Some(-2), - // // Some(-2), - // // Some(-2), - // // Some(-2), - // // Some(-2), - // // Some(-2), - // // Some(-2), - // // ]), - // // )?; - // // Ok(()) - // // } + fn test_i32_result( + expr: NthValue, + partition_evaluator_args: PartitionEvaluatorArgs, + expected: Int32Array, + ) -> Result<()> { + let arr: ArrayRef = Arc::new(Int32Array::from(vec![1, -2, 3, -4, 5, -6, 7, 8])); + let values = vec![arr]; + let mut ranges: Vec> = vec![]; + for i in 0..8 { + ranges.push(Range { + start: 0, + end: i + 1, + }) + } + let mut evaluator = expr.partition_evaluator(partition_evaluator_args)?; + let result = ranges + .iter() + .map(|range| evaluator.evaluate(&values, range)) + .collect::>>()?; + let result = ScalarValue::iter_to_array(result.into_iter())?; + let result = as_int32_array(&result)?; + assert_eq!(expected, *result); + Ok(()) + } + + #[test] + fn first_value() -> Result<()> { + let expr = Arc::new(Column::new("c3", 0)) as Arc; + test_i32_result( + NthValue::first_value(), + PartitionEvaluatorArgs::new(&[expr], &[DataType::Int32], false, false), + Int32Array::from(vec![1; 8]).iter().collect::(), + ) + } + + #[test] + fn last_value() -> Result<()> { + let expr = Arc::new(Column::new("c3", 0)) as Arc; + test_i32_result( + NthValue::last_value(), + PartitionEvaluatorArgs::new(&[expr], &[DataType::Int32], false, false), + Int32Array::from(vec![ + Some(1), + Some(-2), + Some(3), + Some(-4), + Some(5), + Some(-6), + Some(7), + Some(8), + ]), + ) + } + + #[test] + fn nth_value_1() -> Result<()> { + let expr = Arc::new(Column::new("c3", 0)) as Arc; + let n_value = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + + test_i32_result( + NthValue::nth_value(), + PartitionEvaluatorArgs::new( + &[expr, n_value], + &[DataType::Int32], + false, + false, + ), + Int32Array::from(vec![1; 8]), + )?; + Ok(()) + } + + #[test] + fn nth_value_2() -> Result<()> { + let expr = Arc::new(Column::new("c3", 0)) as Arc; + let n_value = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + + test_i32_result( + NthValue::nth_value(), + PartitionEvaluatorArgs::new( + &[expr, n_value], + &[DataType::Int32], + false, + false, + ), + Int32Array::from(vec![ + None, + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + ]), + )?; + Ok(()) + } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 228f20d98b9a..dfc30e809108 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -117,11 +117,10 @@ pub struct ListingTableScanNode { pub target_partitions: u32, #[prost(message, repeated, tag = "13")] pub file_sort_order: ::prost::alloc::vec::Vec, - #[prost( - oneof = "listing_table_scan_node::FileFormatType", - tags = "10, 11, 12, 15" - )] - pub file_format_type: ::core::option::Option, + #[prost(oneof = "listing_table_scan_node::FileFormatType", tags = "10, 11, 12, 15")] + pub file_format_type: ::core::option::Option< + listing_table_scan_node::FileFormatType, + >, } /// Nested message and enum types in `ListingTableScanNode`. pub mod listing_table_scan_node { @@ -257,8 +256,10 @@ pub struct CreateExternalTableNode { #[prost(message, optional, tag = "12")] pub constraints: ::core::option::Option, #[prost(map = "string, message", tag = "13")] - pub column_defaults: - ::std::collections::HashMap<::prost::alloc::string::String, LogicalExprNode>, + pub column_defaults: ::std::collections::HashMap< + ::prost::alloc::string::String, + LogicalExprNode, + >, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct PrepareNode { @@ -744,6 +745,8 @@ pub struct WindowExprNode { pub mod window_expr_node { #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum WindowFunction { + #[prost(enumeration = "super::BuiltInWindowFunction", tag = "2")] + BuiltInFunction(i32), #[prost(string, tag = "3")] Udaf(::prost::alloc::string::String), #[prost(string, tag = "9")] @@ -950,7 +953,9 @@ pub struct FullTableReference { #[derive(Clone, PartialEq, ::prost::Message)] pub struct TableReference { #[prost(oneof = "table_reference::TableReferenceEnum", tags = "1, 2, 3")] - pub table_reference_enum: ::core::option::Option, + pub table_reference_enum: ::core::option::Option< + table_reference::TableReferenceEnum, + >, } /// Nested message and enum types in `TableReference`. pub mod table_reference { @@ -1068,8 +1073,9 @@ pub struct JsonSink { #[prost(message, optional, tag = "1")] pub config: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub writer_options: - ::core::option::Option, + pub writer_options: ::core::option::Option< + super::datafusion_common::JsonWriterOptions, + >, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct JsonSinkExecNode { @@ -1087,8 +1093,9 @@ pub struct CsvSink { #[prost(message, optional, tag = "1")] pub config: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub writer_options: - ::core::option::Option, + pub writer_options: ::core::option::Option< + super::datafusion_common::CsvWriterOptions, + >, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct CsvSinkExecNode { @@ -1106,8 +1113,9 @@ pub struct ParquetSink { #[prost(message, optional, tag = "1")] pub config: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub parquet_options: - ::core::option::Option, + pub parquet_options: ::core::option::Option< + super::datafusion_common::TableParquetOptions, + >, } #[derive(Clone, PartialEq, ::prost::Message)] pub struct ParquetSinkExecNode { @@ -1227,8 +1235,9 @@ pub struct PhysicalAggregateExprNode { #[prost(bytes = "vec", optional, tag = "7")] pub fun_definition: ::core::option::Option<::prost::alloc::vec::Vec>, #[prost(oneof = "physical_aggregate_expr_node::AggregateFunction", tags = "4")] - pub aggregate_function: - ::core::option::Option, + pub aggregate_function: ::core::option::Option< + physical_aggregate_expr_node::AggregateFunction, + >, } /// Nested message and enum types in `PhysicalAggregateExprNode`. pub mod physical_aggregate_expr_node { @@ -1253,13 +1262,16 @@ pub struct PhysicalWindowExprNode { #[prost(bytes = "vec", optional, tag = "9")] pub fun_definition: ::core::option::Option<::prost::alloc::vec::Vec>, #[prost(oneof = "physical_window_expr_node::WindowFunction", tags = "2, 3")] - pub window_function: - ::core::option::Option, + pub window_function: ::core::option::Option< + physical_window_expr_node::WindowFunction, + >, } /// Nested message and enum types in `PhysicalWindowExprNode`. pub mod physical_window_expr_node { #[derive(Clone, PartialEq, ::prost::Oneof)] pub enum WindowFunction { + #[prost(enumeration = "super::BuiltInWindowFunction", tag = "2")] + BuiltInFunction(i32), #[prost(string, tag = "3")] UserDefinedAggrFunction(::prost::alloc::string::String), } @@ -1771,7 +1783,9 @@ pub struct PartitionedFile { #[prost(uint64, tag = "3")] pub last_modified_ns: u64, #[prost(message, repeated, tag = "4")] - pub partition_values: ::prost::alloc::vec::Vec, + pub partition_values: ::prost::alloc::vec::Vec< + super::datafusion_common::ScalarValue, + >, #[prost(message, optional, tag = "5")] pub range: ::core::option::Option, #[prost(message, optional, tag = "6")] @@ -1795,9 +1809,7 @@ pub struct PartitionStats { #[prost(message, repeated, tag = "4")] pub column_stats: ::prost::alloc::vec::Vec, } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum BuiltInWindowFunction { /// @@ -1838,9 +1850,7 @@ impl BuiltInWindowFunction { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum WindowFrameUnits { Rows = 0, @@ -1869,9 +1879,7 @@ impl WindowFrameUnits { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum WindowFrameBoundType { CurrentRow = 0, @@ -1900,9 +1908,7 @@ impl WindowFrameBoundType { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum DateUnit { Day = 0, @@ -1928,9 +1934,7 @@ impl DateUnit { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum InsertOp { Append = 0, @@ -1959,9 +1963,7 @@ impl InsertOp { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum PartitionMode { CollectLeft = 0, @@ -1990,9 +1992,7 @@ impl PartitionMode { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum StreamPartitionMode { SinglePartition = 0, @@ -2018,9 +2018,7 @@ impl StreamPartitionMode { } } } -#[derive( - Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration, -)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum AggregateMode { Partial = 0, diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index a569119f7d08..cb7255bb7873 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -23,20 +23,16 @@ use datafusion_common::{ DFSchema, Dependency, Result, }; use datafusion_expr::expr::WildcardOptions; +use datafusion_expr::expr::{ScalarFunction, Unnest}; use datafusion_expr::planner::PlannerResult; use datafusion_expr::{ expr, Expr, ExprFunctionExt, ExprSchemable, WindowFrame, WindowFunctionDefinition, }; -use datafusion_expr::{ - expr::{ScalarFunction, Unnest}, - BuiltInWindowFunction, -}; use sqlparser::ast::{ DuplicateTreatment, Expr as SQLExpr, Function as SQLFunction, FunctionArg, FunctionArgExpr, FunctionArgumentClause, FunctionArgumentList, FunctionArguments, NullTreatment, ObjectName, OrderByExpr, WindowType, }; -use strum::IntoEnumIterator; /// Suggest a valid function based on an invalid input function name /// @@ -52,7 +48,6 @@ pub fn suggest_valid_function( let mut funcs = Vec::new(); funcs.extend(ctx.udaf_names()); - funcs.extend(BuiltInWindowFunction::iter().map(|func| func.to_string())); funcs.extend(ctx.udwf_names()); funcs diff --git a/datafusion/sqllogictest/test_files/string/large_string.slt.temp b/datafusion/sqllogictest/test_files/string/large_string.slt.temp deleted file mode 100644 index fa65714c6628..000000000000 --- a/datafusion/sqllogictest/test_files/string/large_string.slt.temp +++ /dev/null @@ -1,65 +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. - -include ./init_data.slt.part - -# -------------------------------------- -# Setup test tables with different physical string types -# and repeat tests in `string_query.slt.part` -# -------------------------------------- -statement ok -create table test_basic_operator as -select - arrow_cast(column1, 'LargeUtf8') as ascii_1, - arrow_cast(column2, 'LargeUtf8') as ascii_2, - arrow_cast(column3, 'LargeUtf8') as unicode_1, - arrow_cast(column4, 'LargeUtf8') as unicode_2 -from test_source; - -statement ok -create table test_substr as -select arrow_cast(col1, 'LargeUtf8') as c1 from test_substr_base; - -# select -query TTTT -SELECT ascii_1, ascii_2, unicode_1, unicode_2 FROM test_basic_operator ----- -Andrew X datafusion📊🔥 🔥 -Xiangpeng Xiangpeng datafusion数据融合 datafusion数据融合 -Raphael R datafusionДатаФусион аФус -NULL R NULL 🔥 - -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see detail: https://github.com/apache/datafusion/issues/12637 -# Test pattern with wildcard characters -query TTBBBB -select ascii_1, unicode_1, - ascii_1 like 'An%' as ascii_like, - unicode_1 like '%ion数据%' as unicode_like, - ascii_1 ilike 'An%' as ascii_ilike, - unicode_1 ilike '%ion数据%' as unicode_ilik -from test_basic_operator; ----- -Andrew datafusion📊🔥 true false true false -Xiangpeng datafusion数据融合 false true false true -Raphael datafusionДатаФусион false false false false -NULL NULL NULL NULL NULL NULL - -# -# common test for string-like functions and operators -# -include ./string_query.slt.part diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt.temp b/datafusion/sqllogictest/test_files/string/string_view.slt.temp deleted file mode 100644 index 183ce75e0667..000000000000 --- a/datafusion/sqllogictest/test_files/string/string_view.slt.temp +++ /dev/null @@ -1,43 +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. - -include ./init_data.slt.part - -# -------------------------------------- -# Setup test tables with different physical string types -# and repeat tests in `string_query.slt.part` -# -------------------------------------- -statement ok -create table test_basic_operator as -select - arrow_cast(column1, 'Utf8View') as ascii_1, - arrow_cast(column2, 'Utf8View') as ascii_2, - arrow_cast(column3, 'Utf8View') as unicode_1, - arrow_cast(column4, 'Utf8View') as unicode_2 -from test_source; - -statement ok -create table test_substr as -select arrow_cast(col1, 'Utf8View') as c1 from test_substr_base; - -statement ok -drop table test_source - -# -# common test for string-like functions and operators -# -include ./string_query.slt.part From 7fb84fa1f8aedd4b37138dc2f9f71ed4d2a3d626 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Sun, 3 Nov 2024 23:28:56 +0300 Subject: [PATCH 03/18] test --- datafusion/sqllogictest/test_files/errors.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index da46a7e5e679..edd31e3ef0f4 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -120,7 +120,7 @@ from aggregate_test_100 order by c9 # WindowFunction with BuiltInWindowFunction wrong signature -statement error DataFusion error: Error during planning: No function matches the given name and argument types 'NTH_VALUE\(Int32, Int64, Int64\)'\. You might need to add explicit type casts\.\n\tCandidate functions:\n\tNTH_VALUE\(Any, Any\) +statement error Coercion select c9, nth_value(c5, 2, 3) over (order by c9) as nv1 From e6ef6ed7ba0cd18c91d2cb1d6b4d441af14d470f Mon Sep 17 00:00:00 2001 From: buraksenb Date: Mon, 4 Nov 2024 09:26:10 +0300 Subject: [PATCH 04/18] proto and rustlint --- datafusion/functions-window/src/nth_value.rs | 20 +-- .../src/windows/bounded_window_agg_exec.rs | 131 +----------------- datafusion/proto/proto/datafusion.proto | 11 -- datafusion/proto/src/generated/pbjson.rs | 9 -- datafusion/proto/src/generated/prost.rs | 17 --- .../proto/src/logical_plan/from_proto.rs | 10 +- datafusion/proto/src/logical_plan/to_proto.rs | 16 +-- .../proto/src/physical_plan/from_proto.rs | 1 + .../proto/src/physical_plan/to_proto.rs | 39 +----- 9 files changed, 23 insertions(+), 231 deletions(-) diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index 5e36c15ce91d..512645e72f2b 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -43,19 +43,19 @@ define_udwf_and_expr!( First, first_value, "returns the first value in the window frame", - NthValue::first_value + NthValue::first ); define_udwf_and_expr!( Last, last_value, "returns the last value in the window frame", - NthValue::last_value + NthValue::last ); define_udwf_and_expr!( NthValue, nth_value, "returns the nth value in the window frame", - NthValue::nth_value + NthValue::nth ); /// Tag to differentiate special use cases of the NTH_VALUE built-in window function. @@ -98,14 +98,14 @@ impl NthValue { } } - pub fn first_value() -> Self { + pub fn first() -> Self { Self::new(NthValueKind::First) } - pub fn last_value() -> Self { + pub fn last() -> Self { Self::new(NthValueKind::Last) } - pub fn nth_value() -> Self { + pub fn nth() -> Self { Self::new(NthValueKind::Nth) } } @@ -447,7 +447,7 @@ mod tests { fn first_value() -> Result<()> { let expr = Arc::new(Column::new("c3", 0)) as Arc; test_i32_result( - NthValue::first_value(), + NthValue::first(), PartitionEvaluatorArgs::new(&[expr], &[DataType::Int32], false, false), Int32Array::from(vec![1; 8]).iter().collect::(), ) @@ -457,7 +457,7 @@ mod tests { fn last_value() -> Result<()> { let expr = Arc::new(Column::new("c3", 0)) as Arc; test_i32_result( - NthValue::last_value(), + NthValue::last(), PartitionEvaluatorArgs::new(&[expr], &[DataType::Int32], false, false), Int32Array::from(vec![ Some(1), @@ -479,7 +479,7 @@ mod tests { Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; test_i32_result( - NthValue::nth_value(), + NthValue::nth(), PartitionEvaluatorArgs::new( &[expr, n_value], &[DataType::Int32], @@ -498,7 +498,7 @@ mod tests { Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; test_i32_result( - NthValue::nth_value(), + NthValue::nth(), PartitionEvaluatorArgs::new( &[expr, n_value], &[DataType::Int32], 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 8c0331f94570..15e524982d6a 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1158,9 +1158,7 @@ mod tests { use std::task::{Context, Poll}; use std::time::Duration; - use crate::common::collect; use crate::expressions::PhysicalSortExpr; - use crate::memory::MemoryExec; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; use crate::windows::{create_window_expr, BoundedWindowAggExec, InputOrderMode}; @@ -1180,13 +1178,9 @@ mod tests { WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::expressions::{col, Column, NthValue}; - use datafusion_physical_expr::window::{ - BuiltInWindowExpr, BuiltInWindowFunctionExpr, - }; + use datafusion_physical_expr::expressions::{col, Column}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; - use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1500,129 +1494,6 @@ mod tests { Ok(source) } - // Tests NTH_VALUE(negative index) with memoize feature. - // To be able to trigger memoize feature for NTH_VALUE we need to - // - feed BoundedWindowAggExec with batch stream data. - // - Window frame should contain UNBOUNDED PRECEDING. - // It hard to ensure these conditions are met, from the sql query. - #[tokio::test] - async fn test_window_nth_value_bounded_memoize() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); - - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - // Create a new batch of data to insert into the table - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], - )?; - - let memory_exec = MemoryExec::try_new( - &[vec![batch.clone(), batch.clone(), batch.clone()]], - Arc::clone(&schema), - None, - ) - .map(|e| Arc::new(e) as Arc)?; - let col_a = col("a", &schema)?; - let nth_value_func1 = NthValue::nth( - "nth_value(-1)", - Arc::clone(&col_a), - DataType::Int32, - 1, - false, - )? - .reverse_expr() - .unwrap(); - let nth_value_func2 = NthValue::nth( - "nth_value(-2)", - Arc::clone(&col_a), - DataType::Int32, - 2, - false, - )? - .reverse_expr() - .unwrap(); - let last_value_func = Arc::new(NthValue::last( - "last", - Arc::clone(&col_a), - DataType::Int32, - false, - )) as _; - let window_exprs = vec![ - // LAST_VALUE(a) - Arc::new(BuiltInWindowExpr::new( - last_value_func, - &[], - LexOrderingRef::default(), - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - // NTH_VALUE(a, -1) - Arc::new(BuiltInWindowExpr::new( - nth_value_func1, - &[], - LexOrderingRef::default(), - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - // NTH_VALUE(a, -2) - Arc::new(BuiltInWindowExpr::new( - nth_value_func2, - &[], - LexOrderingRef::default(), - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - ]; - let physical_plan = BoundedWindowAggExec::try_new( - window_exprs, - memory_exec, - vec![], - InputOrderMode::Sorted, - ) - .map(|e| Arc::new(e) as Arc)?; - - let batches = collect(physical_plan.execute(0, task_ctx)?).await?; - - let expected = vec![ - "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[3]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+------+---------------+---------------+", - "| a | last | nth_value(-1) | nth_value(-2) |", - "+---+------+---------------+---------------+", - "| 1 | 1 | 1 | |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "| 1 | 1 | 1 | 3 |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "| 1 | 1 | 1 | 3 |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "+---+------+---------------+---------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - // This test, tests whether most recent row guarantee by the input batch of the `BoundedWindowAggExec` // helps `BoundedWindowAggExec` to generate low latency result in the `Linear` mode. // Input data generated at the source is diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index b68c47c57eb9..23dc84579b54 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -507,17 +507,6 @@ message ScalarUDFExprNode { enum BuiltInWindowFunction { UNSPECIFIED = 0; // https://protobuf.dev/programming-guides/dos-donts/#unspecified-enum - // ROW_NUMBER = 0; - // RANK = 1; - // DENSE_RANK = 2; - // PERCENT_RANK = 3; - // CUME_DIST = 4; - // NTILE = 5; - // LAG = 6; - // LEAD = 7; - FIRST_VALUE = 8; - LAST_VALUE = 9; - NTH_VALUE = 10; } message WindowExprNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index e54edb718808..549dc660432a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -1662,9 +1662,6 @@ impl serde::Serialize for BuiltInWindowFunction { { let variant = match self { Self::Unspecified => "UNSPECIFIED", - Self::FirstValue => "FIRST_VALUE", - Self::LastValue => "LAST_VALUE", - Self::NthValue => "NTH_VALUE", }; serializer.serialize_str(variant) } @@ -1677,9 +1674,6 @@ impl<'de> serde::Deserialize<'de> for BuiltInWindowFunction { { const FIELDS: &[&str] = &[ "UNSPECIFIED", - "FIRST_VALUE", - "LAST_VALUE", - "NTH_VALUE", ]; struct GeneratedVisitor; @@ -1721,9 +1715,6 @@ impl<'de> serde::Deserialize<'de> for BuiltInWindowFunction { { match value { "UNSPECIFIED" => Ok(BuiltInWindowFunction::Unspecified), - "FIRST_VALUE" => Ok(BuiltInWindowFunction::FirstValue), - "LAST_VALUE" => Ok(BuiltInWindowFunction::LastValue), - "NTH_VALUE" => Ok(BuiltInWindowFunction::NthValue), _ => 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 dfc30e809108..1d211554a327 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1814,17 +1814,6 @@ pub struct PartitionStats { pub enum BuiltInWindowFunction { /// Unspecified = 0, - /// ROW_NUMBER = 0; - /// RANK = 1; - /// DENSE_RANK = 2; - /// PERCENT_RANK = 3; - /// CUME_DIST = 4; - /// NTILE = 5; - /// LAG = 6; - /// LEAD = 7; - FirstValue = 8, - LastValue = 9, - NthValue = 10, } impl BuiltInWindowFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -1834,18 +1823,12 @@ impl BuiltInWindowFunction { pub fn as_str_name(&self) -> &'static str { match self { Self::Unspecified => "UNSPECIFIED", - Self::FirstValue => "FIRST_VALUE", - Self::LastValue => "LAST_VALUE", - Self::NthValue => "NTH_VALUE", } } /// Creates an enum from field names used in the ProtoBuf definition. pub fn from_str_name(value: &str) -> ::core::option::Option { match value { "UNSPECIFIED" => Some(Self::Unspecified), - "FIRST_VALUE" => Some(Self::FirstValue), - "LAST_VALUE" => Some(Self::LastValue), - "NTH_VALUE" => Some(Self::NthValue), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 500c4bbdfad1..b8107b4045c1 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -148,13 +148,8 @@ impl From<&protobuf::StringifiedPlan> for StringifiedPlan { } impl From for BuiltInWindowFunction { - fn from(built_in_function: protobuf::BuiltInWindowFunction) -> Self { - match built_in_function { - protobuf::BuiltInWindowFunction::Unspecified => todo!(), - protobuf::BuiltInWindowFunction::FirstValue => Self::FirstValue, - protobuf::BuiltInWindowFunction::NthValue => Self::NthValue, - protobuf::BuiltInWindowFunction::LastValue => Self::LastValue, - } + fn from(_built_in_function: protobuf::BuiltInWindowFunction) -> Self { + unreachable!() } } @@ -287,6 +282,7 @@ pub fn parse_expr( // TODO: support proto for null treatment match window_function { + window_expr_node::WindowFunction::BuiltInFunction(_) => unreachable!(), window_expr_node::WindowFunction::Udaf(udaf_name) => { let udaf_function = match &expr.fun_definition { Some(buf) => codec.try_decode_udaf(udaf_name, buf)?, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 247bcd23bc1f..6628ffbce4d5 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -25,9 +25,9 @@ use datafusion_expr::expr::{ ScalarFunction, Unnest, }; use datafusion_expr::{ - logical_plan::PlanType, logical_plan::StringifiedPlan, BuiltInWindowFunction, Expr, - JoinConstraint, JoinType, SortExpr, TryCast, WindowFrame, WindowFrameBound, - WindowFrameUnits, WindowFunctionDefinition, + logical_plan::PlanType, logical_plan::StringifiedPlan, Expr, JoinConstraint, + JoinType, SortExpr, TryCast, WindowFrame, WindowFrameBound, WindowFrameUnits, + WindowFunctionDefinition, }; use crate::protobuf::RecursionUnnestOption; @@ -121,16 +121,6 @@ impl From<&StringifiedPlan> for protobuf::StringifiedPlan { } } -impl From<&BuiltInWindowFunction> for protobuf::BuiltInWindowFunction { - fn from(value: &BuiltInWindowFunction) -> Self { - match value { - BuiltInWindowFunction::FirstValue => Self::FirstValue, - BuiltInWindowFunction::LastValue => Self::LastValue, - BuiltInWindowFunction::NthValue => Self::NthValue, - } - } -} - impl From for protobuf::WindowFrameUnits { fn from(units: WindowFrameUnits) -> Self { match units { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 417e4069cb31..be7d4e5146af 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -146,6 +146,7 @@ pub fn parse_physical_window_expr( let fun = if let Some(window_func) = proto.window_function.as_ref() { match window_func { + protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(_) => unreachable!(), protobuf::physical_window_expr_node::WindowFunction::UserDefinedAggrFunction(udaf_name) => { WindowFunctionDefinition::AggregateUDF(match &proto.fun_definition { Some(buf) => codec.try_decode_udaf(udaf_name, buf)?, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 4bf7e353326e..5e8794b6d617 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -20,14 +20,14 @@ use std::sync::Arc; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; -use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; +use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, - Literal, NegativeExpr, NotExpr, NthValue, TryCastExpr, + Literal, NegativeExpr, NotExpr, TryCastExpr, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; -use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; +use datafusion::physical_plan::windows::PlainAggregateWindowExpr; use datafusion::physical_plan::{Partitioning, PhysicalExpr, WindowExpr}; use datafusion::{ datasource::{ @@ -102,39 +102,10 @@ pub fn serialize_physical_window_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let expr = window_expr.as_any(); - let mut args = window_expr.expressions().to_vec(); + let args = window_expr.expressions().to_vec(); let window_frame = window_expr.get_window_frame(); - let (window_function, fun_definition) = if let Some(built_in_window_expr) = - expr.downcast_ref::() - { - let expr = built_in_window_expr.get_built_in_func_expr(); - let built_in_fn_expr = expr.as_any(); - - let builtin_fn = - if let Some(nth_value_expr) = built_in_fn_expr.downcast_ref::() { - match nth_value_expr.get_kind() { - NthValueKind::First => protobuf::BuiltInWindowFunction::FirstValue, - NthValueKind::Last => protobuf::BuiltInWindowFunction::LastValue, - NthValueKind::Nth(n) => { - args.insert( - 1, - Arc::new(Literal::new( - datafusion_common::ScalarValue::Int64(Some(n)), - )), - ); - protobuf::BuiltInWindowFunction::NthValue - } - } - } else { - return not_impl_err!("BuiltIn function not supported: {expr:?}"); - }; - - ( - physical_window_expr_node::WindowFunction::BuiltInFunction(builtin_fn as i32), - None, - ) - } else if let Some(plain_aggr_window_expr) = + let (window_function, fun_definition) = if let Some(plain_aggr_window_expr) = expr.downcast_ref::() { serialize_physical_window_aggr_expr( From 1131df0d4526ea4b093b2dafc8f11b1aabce2486 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Mon, 4 Nov 2024 12:14:06 +0300 Subject: [PATCH 05/18] fix datatype --- datafusion/functions-window/src/nth_value.rs | 35 ++++++++++++------- datafusion/sqllogictest/test_files/window.slt | 24 ++++++------- 2 files changed, 34 insertions(+), 25 deletions(-) diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index 512645e72f2b..ca5f7e5abc2b 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -27,7 +27,7 @@ use std::sync::OnceLock; use datafusion_common::arrow::array::ArrayRef; use datafusion_common::arrow::datatypes::{DataType, Field}; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::window_doc_sections::DOC_SECTION_ANALYTICAL; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::{ @@ -36,7 +36,6 @@ use datafusion_expr::{ }; use datafusion_functions_window_common::field; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; - use field::WindowUDFFieldArgs; define_udwf_and_expr!( @@ -188,20 +187,29 @@ impl WindowUDFImpl for NthValue { kind: self.kind, }; - let n: i64 = if matches!(self.kind, NthValueKind::Nth) { - get_scalar_value_from_args(partition_evaluator_args.input_exprs(), 1)? + if !matches!(self.kind, NthValueKind::Nth) { + return Ok(Box::new(NthValueEvaluator { + state, + ignore_nulls: partition_evaluator_args.ignore_nulls(), + n: 0, + })); + } + + let n = + match get_scalar_value_from_args(partition_evaluator_args.input_exprs(), 1)? .map(get_signed_integer) - .map_or(Ok(0), |v| v) - .map(|val| { + { + Some(Ok(n)) => { if partition_evaluator_args.is_reversed() { - -val + Ok(-n) } else { - val + Ok(n) } - })? - } else { - 0 - }; + } + _ => { + exec_err!("Expected an integer value") + } + }?; Ok(Box::new(NthValueEvaluator { state, @@ -212,8 +220,9 @@ impl WindowUDFImpl for NthValue { fn field(&self, field_args: WindowUDFFieldArgs) -> Result { let nullable = true; + let return_type = field_args.input_types().first().unwrap_or(&DataType::Null); - Ok(Field::new(field_args.name(), DataType::UInt64, nullable)) + Ok(Field::new(field_args.name(), return_type.clone(), nullable)) } fn reverse_expr(&self) -> ReversedUDWF { diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index d593a985c458..8509268a0bfc 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2660,14 +2660,14 @@ EXPLAIN SELECT ---- logical_plan 01)Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 -02)--Projection: annotated_data_finite.ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 -03)----WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, lag(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, lag(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, lead(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, lead(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +02)--Projection: annotated_data_finite.ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 +03)----WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, nth_value(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, nth_value(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, lag(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, lag(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, lead(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, lead(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] 04)------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, lag(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, lag(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, lead(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, lead(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] 05)--------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan 01)SortExec: TopK(fetch=5), expr=[ts@0 DESC], preserve_partitioning=[false] -02)--ProjectionExec: expr=[ts@0 as ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] -03)----BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(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 }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) 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 }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, 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 }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, 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 }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, 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 }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, 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 }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, 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 }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, 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 }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 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 }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 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 }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }], mode=[Sorted] +02)--ProjectionExec: expr=[ts@0 as ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] +03)----BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(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 }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) 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 }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, 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 }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, 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 }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, 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 }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, 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 }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, 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 }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, 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 }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 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 }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 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 }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 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 }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(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 }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 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 }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 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 }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 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 }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 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 }], mode=[Sorted] 05)--------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 @@ -2783,15 +2783,15 @@ EXPLAIN SELECT logical_plan 01)Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 02)--Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -03)----Projection: first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS nth_value1, annotated_data_finite.inc_col -04)------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +03)----Projection: first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS last_value2, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS nth_value1, annotated_data_finite.inc_col +04)------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, nth_value(annotated_data_finite.inc_col, Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] 05)--------WindowAggr: windowExpr=[[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] 06)----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan 01)ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1] 02)--SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST], preserve_partitioning=[false] -03)----ProjectionExec: expr=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] -04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, 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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, 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 }, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, 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] +03)----ProjectionExec: expr=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] +04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, 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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, 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 }, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, 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=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, 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 }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, 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)----------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 @@ -3565,13 +3565,13 @@ EXPLAIN SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 ---- logical_plan 01)Sort: multiple_ordered_table.c ASC NULLS LAST, fetch=5 -02)--Projection: multiple_ordered_table.c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS nv1 -03)----WindowAggr: windowExpr=[[NTH_VALUE(multiple_ordered_table.c, Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +02)--Projection: multiple_ordered_table.c, nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS nv1 +03)----WindowAggr: windowExpr=[[nth_value(multiple_ordered_table.c, Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: multiple_ordered_table projection=[c] physical_plan -01)ProjectionExec: expr=[c@0 as c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] +01)ProjectionExec: expr=[c@0 as c, nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----WindowAggExec: wdw=[NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] +03)----WindowAggExec: wdw=[nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true query II From f28163a1986654049432ca9ba556456f5b541cfc Mon Sep 17 00:00:00 2001 From: buraksenb Date: Mon, 4 Nov 2024 12:31:00 +0300 Subject: [PATCH 06/18] cont --- datafusion/functions-window/src/nth_value.rs | 37 +++++++++++++++++-- datafusion/sqllogictest/test_files/window.slt | 2 +- 2 files changed, 34 insertions(+), 5 deletions(-) diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index ca5f7e5abc2b..14f17d332200 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -201,15 +201,17 @@ impl WindowUDFImpl for NthValue { { Some(Ok(n)) => { if partition_evaluator_args.is_reversed() { - Ok(-n) + -n } else { - Ok(n) + n } } _ => { - exec_err!("Expected an integer value") + return exec_err!( + "Expected a signed integer literal for the second argument of nth_value" + ) } - }?; + }; Ok(Box::new(NthValueEvaluator { state, @@ -527,4 +529,31 @@ mod tests { )?; Ok(()) } + + #[test] + fn nth_value_fail() -> Result<()> { + let expr = Arc::new(Column::new("c3", 0)) as Arc; + let n_value = Arc::new(Column::new("c4", 0)) as Arc; + + test_i32_result( + NthValue::nth(), + PartitionEvaluatorArgs::new( + &[expr, n_value], + &[DataType::Int32], + false, + false, + ), + Int32Array::from(vec![ + None, + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + ]), + )?; + Ok(()) + } } diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8509268a0bfc..a00c470e05fe 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4892,7 +4892,7 @@ DROP TABLE t1; statement ok CREATE TABLE t1(v1 BIGINT); -query error DataFusion error: Execution error: Expected a signed integer literal for the second argument of nth_value, got v1@0 +query error DataFusion error: Execution error: Expected a signed integer literal for the second argument of nth_value SELECT NTH_VALUE('+Inf'::Double, v1) OVER (PARTITION BY v1) FROM t1; statement ok From fddbc58d53a7ecf65d6803c4844db61a7f84a173 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Mon, 4 Nov 2024 12:32:08 +0300 Subject: [PATCH 07/18] cont --- datafusion/functions-window/src/nth_value.rs | 27 ------------------- datafusion/sqllogictest/test_files/window.slt | 2 +- 2 files changed, 1 insertion(+), 28 deletions(-) diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index 14f17d332200..2fcf82eeef25 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -529,31 +529,4 @@ mod tests { )?; Ok(()) } - - #[test] - fn nth_value_fail() -> Result<()> { - let expr = Arc::new(Column::new("c3", 0)) as Arc; - let n_value = Arc::new(Column::new("c4", 0)) as Arc; - - test_i32_result( - NthValue::nth(), - PartitionEvaluatorArgs::new( - &[expr, n_value], - &[DataType::Int32], - false, - false, - ), - Int32Array::from(vec![ - None, - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - ]), - )?; - Ok(()) - } } diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index a00c470e05fe..8e3559a32684 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4864,7 +4864,7 @@ SELECT lag(column2, 1.1) OVER (order by column1) FROM t; query error DataFusion error: Execution error: Expected an integer value SELECT lead(column2, 1.1) OVER (order by column1) FROM t; -query error DataFusion error: Execution error: Expected an integer value +query error DataFusion error: Execution error: Expected a signed integer literal for the second argument of nth_value SELECT nth_value(column2, 1.1) OVER (order by column1) FROM t; statement ok From c4065058e3449f59753176d5629759a6ecd61d74 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Tue, 5 Nov 2024 08:50:58 +0300 Subject: [PATCH 08/18] apply jcsherins early validation --- datafusion/physical-plan/src/windows/mod.rs | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index b592f5cf8787..c5a40518f065 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -168,14 +168,31 @@ fn create_udwf_window_expr( .map(|arg| arg.data_type(input_schema)) .collect::>()?; - Ok(Arc::new(WindowUDFExpr { + let udwf_expr = Arc::new(WindowUDFExpr { fun: Arc::clone(fun), args: args.to_vec(), input_types, name, is_reversed: false, ignore_nulls, - })) + }); + + /// Early validation of input expressions + /// + /// We create a partition evaluator because in the user-defined window + /// implementation this is where code for parsing input expressions + /// exist. The benefits are: + /// - If any of the input expressions are invalid we catch them early + /// in the planning phase, rather than during execution. + /// - Maintains compatibility with built-in (now removed) window + /// functions validation behavior. + /// - Predictable and reliable error handling. + /// + /// See discussion here: + /// https://github.com/apache/datafusion/pull/13201#issuecomment-2454209975 + let _ = udwf_expr.create_evaluator()?; + + Ok(udwf_expr) } /// Implements [`BuiltInWindowFunctionExpr`] for [`WindowUDF`] From 12c0a5255be29c52b8a0378b425983cf35dd1f4e Mon Sep 17 00:00:00 2001 From: buraksenb Date: Tue, 5 Nov 2024 09:26:07 +0300 Subject: [PATCH 09/18] docs --- datafusion/expr/src/expr.rs | 7 +++--- datafusion/physical-plan/src/windows/mod.rs | 24 +++++++++---------- datafusion/sqllogictest/test_files/window.slt | 2 +- 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index c404dec898bd..0e2dfdb311eb 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -764,10 +764,11 @@ impl From> for WindowFunctionDefinition { /// ``` /// # use datafusion_expr::{Expr, BuiltInWindowFunction, col, ExprFunctionExt}; /// # use datafusion_expr::expr::WindowFunction; -/// use datafusion_expr::WindowFunctionDefinition::WindowUDF; -/// // Create FIRST_VALUE(a) OVER (PARTITION BY b ORDER BY c) +/// use datafusion_expr::test::function_stub::count_udaf; +/// use datafusion_expr::WindowFunctionDefinition::{AggregateUDF}; +/// // Create COUNT(a) OVER (PARTITION BY b ORDER BY c) /// let expr = Expr::WindowFunction( -/// WindowFunction::new(WindowUDF::, vec![col("a")]) +/// WindowFunction::new(AggregateUDF(count_udaf()), vec![col("a")]) /// ) /// .partition_by(vec![col("b")]) /// .order_by(vec![col("b").sort(true, true)]) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index c5a40518f065..ea2ec84c705f 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -177,19 +177,17 @@ fn create_udwf_window_expr( ignore_nulls, }); - /// Early validation of input expressions - /// - /// We create a partition evaluator because in the user-defined window - /// implementation this is where code for parsing input expressions - /// exist. The benefits are: - /// - If any of the input expressions are invalid we catch them early - /// in the planning phase, rather than during execution. - /// - Maintains compatibility with built-in (now removed) window - /// functions validation behavior. - /// - Predictable and reliable error handling. - /// - /// See discussion here: - /// https://github.com/apache/datafusion/pull/13201#issuecomment-2454209975 + // Early validation of input expressions + // We create a partition evaluator because in the user-defined window + // implementation this is where code for parsing input expressions + // exist. The benefits are: + // - If any of the input expressions are invalid we catch them early + // in the planning phase, rather than during execution. + // - Maintains compatibility with built-in (now removed) window + // functions validation behavior. + // - Predictable and reliable error handling. + // See discussion here: + // https://github.com/apache/datafusion/pull/13201#issuecomment-2454209975 let _ = udwf_expr.create_evaluator()?; Ok(udwf_expr) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8e3559a32684..5bbe5cfc172a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4892,7 +4892,7 @@ DROP TABLE t1; statement ok CREATE TABLE t1(v1 BIGINT); -query error DataFusion error: Execution error: Expected a signed integer literal for the second argument of nth_value +query error DataFusion error: This feature is not implemented: There is only support Literal types for field at idx: 1 in Window Function SELECT NTH_VALUE('+Inf'::Double, v1) OVER (PARTITION BY v1) FROM t1; statement ok From 6e130a8bbfa488ff9660d5dc062fdffa58be7d77 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Tue, 5 Nov 2024 10:56:44 +0300 Subject: [PATCH 10/18] doc --- .../user-guide/sql/window_functions_new.md | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/docs/source/user-guide/sql/window_functions_new.md b/docs/source/user-guide/sql/window_functions_new.md index ae3edb832fcb..1727dececbeb 100644 --- a/docs/source/user-guide/sql/window_functions_new.md +++ b/docs/source/user-guide/sql/window_functions_new.md @@ -218,8 +218,23 @@ row_number() ## Analytical Functions +- [first_value](#first_value) - [lag](#lag) +- [last_value](#last_value) - [lead](#lead) +- [nth_value](#nth_value) + +### `first_value` + +Returns value evaluated at the row that is the first row of the window frame. + +``` +first_value(expression) +``` + +#### Arguments + +- **expression**: Expression to operate on ### `lag` @@ -235,6 +250,18 @@ lag(expression, offset, default) - **offset**: Integer. Specifies how many rows back the value of expression should be retrieved. Defaults to 1. - **default**: The default value if the offset is not within the partition. Must be of the same type as expression. +### `last_value` + +Returns value evaluated at the row that is the last row of the window frame. + +``` +last_value(expression) +``` + +#### Arguments + +- **expression**: Expression to operate on + ### `lead` Returns value evaluated at the row that is offset rows after the current row within the partition; if there is no such row, instead return default (which must be of the same type as value). @@ -248,3 +275,16 @@ lead(expression, offset, default) - **expression**: Expression to operate on - **offset**: Integer. Specifies how many rows forward the value of expression should be retrieved. Defaults to 1. - **default**: The default value if the offset is not within the partition. Must be of the same type as expression. + +### `nth_value` + +Returns value evaluated at the row that is the nth row of the window frame (counting from 1); null if no such row. + +``` +nth_value(expression, n) +``` + +#### Arguments + +- **expression**: The name the column of which nth value to retrieve +- **n**: Integer. Specifies the n in nth From 837809815a4d1f2ebdff0227b33b4af9292d3f04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Wed, 6 Nov 2024 17:27:09 +0300 Subject: [PATCH 11/18] Apply suggestions from code review Co-authored-by: Sherin Jacob --- datafusion/sqllogictest/test_files/errors.slt | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index edd31e3ef0f4..6bd42e591710 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -120,12 +120,20 @@ from aggregate_test_100 order by c9 # WindowFunction with BuiltInWindowFunction wrong signature -statement error Coercion +statement error select c9, nth_value(c5, 2, 3) over (order by c9) as nv1 from aggregate_test_100 order by c9 +---- +DataFusion error: Error during planning: Error during planning: Coercion from [Int32, Int64, Int64] to the signature OneOf([Any(0), Any(1), Any(2)]) failed. No function matches the given name and argument types 'nth_value(Int32, Int64, Int64)'. You might need to add explicit type casts. + Candidate functions: + nth_value() + nth_value(Any) + nth_value(Any, Any) +from aggregate_test_100 +order by c9 query error DataFusion error: Arrow error: Cast error: Cannot cast string 'foo' to value of Int64 type From 17909a8228d02fdf2cd6e745f8ae8a920691d879 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Sun, 10 Nov 2024 22:27:19 +0300 Subject: [PATCH 12/18] passes lint but does not have tests --- datafusion/core/src/dataframe/mod.rs | 30 +- .../core/tests/fuzz_cases/window_fuzz.rs | 27 ++ .../expr/src/built_in_window_function.rs | 105 ++++- datafusion/expr/src/expr.rs | 26 +- datafusion/expr/src/expr_schema.rs | 6 + datafusion/expr/src/lib.rs | 1 + datafusion/expr/src/window_function.rs | 26 ++ datafusion/functions-window/src/nth_value.rs | 27 +- .../physical-expr/src/window/nth_value.rs | 415 ++++++++++++++++++ .../src/windows/bounded_window_agg_exec.rs | 244 +++++----- datafusion/physical-plan/src/windows/mod.rs | 3 + datafusion/proto/proto/datafusion.proto | 11 + .../proto/src/logical_plan/from_proto.rs | 26 +- datafusion/proto/src/logical_plan/to_proto.rs | 1 + .../proto/src/physical_plan/from_proto.rs | 10 +- .../tests/cases/roundtrip_physical_plan.rs | 39 +- 16 files changed, 821 insertions(+), 176 deletions(-) create mode 100644 datafusion/expr/src/window_function.rs create mode 100644 datafusion/physical-expr/src/window/nth_value.rs diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 9488b5c41663..2c71cb80d755 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1946,8 +1946,9 @@ mod tests { use datafusion_common_runtime::SpawnedTask; use datafusion_expr::expr::WindowFunction; use datafusion_expr::{ - cast, create_udf, lit, ExprFunctionExt, ScalarFunctionImplementation, Volatility, - WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, + cast, create_udf, lit, BuiltInWindowFunction, ExprFunctionExt, + ScalarFunctionImplementation, Volatility, WindowFrame, WindowFrameBound, + WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::expr_fn::{array_agg, count_distinct}; use datafusion_functions_window::expr_fn::row_number; @@ -2171,6 +2172,31 @@ mod tests { Ok(()) } + #[tokio::test] + async fn select_with_window_exprs() -> Result<()> { + // build plan using Table API + let t = test_table().await?; + let first_row = Expr::WindowFunction(WindowFunction::new( + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::FirstValue, + ), + vec![col("aggregate_test_100.c1")], + )) + .partition_by(vec![col("aggregate_test_100.c2")]) + .build() + .unwrap(); + let t2 = t.select(vec![col("c1"), first_row])?; + let plan = t2.plan.clone(); + + let sql_plan = create_plan( + "select c1, first_value(c1) over (partition by c2) from aggregate_test_100", + ) + .await?; + + assert_same_plan(&plan, &sql_plan); + Ok(()) + } + #[tokio::test] async fn select_with_periods() -> Result<()> { // define data with a column name that has a "." in it: diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index dfac26d4374e..eaa84988a85d 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -46,6 +46,9 @@ use test_utils::add_empty_batches; use datafusion::functions_window::row_number::row_number_udwf; use datafusion_common::HashMap; use datafusion_functions_window::lead_lag::{lag_udwf, lead_udwf}; +use datafusion_functions_window::nth_value::{ + first_value_udwf, last_value_udwf, nth_value_udwf, +}; use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use rand::distributions::Alphanumeric; @@ -414,6 +417,30 @@ fn get_random_function( ), ); } + window_fn_map.insert( + "first_value", + ( + WindowFunctionDefinition::WindowUDF(first_value_udwf()), + vec![arg.clone()], + ), + ); + window_fn_map.insert( + "last_value", + ( + WindowFunctionDefinition::WindowUDF(last_value_udwf()), + vec![arg.clone()], + ), + ); + window_fn_map.insert( + "nth_value", + ( + WindowFunctionDefinition::WindowUDF(nth_value_udwf()), + vec![ + arg.clone(), + lit(ScalarValue::Int64(Some(rng.gen_range(1..10)))), + ], + ), + ); let rand_fn_idx = rng.gen_range(0..window_fn_map.len()); let fn_name = window_fn_map.keys().collect::>()[rand_fn_idx]; diff --git a/datafusion/expr/src/built_in_window_function.rs b/datafusion/expr/src/built_in_window_function.rs index b1ba6d239ada..ab41395ad371 100644 --- a/datafusion/expr/src/built_in_window_function.rs +++ b/datafusion/expr/src/built_in_window_function.rs @@ -17,12 +17,115 @@ //! Built-in functions module contains all the built-in functions definitions. +use std::fmt; +use std::str::FromStr; + +use crate::type_coercion::functions::data_types; +use crate::utils; +use crate::{Signature, Volatility}; +use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; + +use arrow::datatypes::DataType; + use strum_macros::EnumIter; +impl fmt::Display for BuiltInWindowFunction { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", self.name()) + } +} + /// A [window function] built in to DataFusion /// /// [Window Function]: https://en.wikipedia.org/wiki/Window_function_(SQL) #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, EnumIter)] pub enum BuiltInWindowFunction { - Stub, + /// returns value evaluated at the row that is the first row of the window frame + FirstValue, + /// Returns value evaluated at the row that is the last row of the window frame + LastValue, + /// Returns value evaluated at the row that is the nth row of the window frame (counting from 1); returns null if no such row + NthValue, +} + +impl BuiltInWindowFunction { + pub fn name(&self) -> &str { + use BuiltInWindowFunction::*; + match self { + FirstValue => "first_value", + LastValue => "last_value", + NthValue => "NTH_VALUE", + } + } +} + +impl FromStr for BuiltInWindowFunction { + type Err = DataFusionError; + fn from_str(name: &str) -> Result { + Ok(match name.to_uppercase().as_str() { + "FIRST_VALUE" => BuiltInWindowFunction::FirstValue, + "LAST_VALUE" => BuiltInWindowFunction::LastValue, + "NTH_VALUE" => BuiltInWindowFunction::NthValue, + _ => return plan_err!("There is no built-in window function named {name}"), + }) + } +} + +/// Returns the datatype of the built-in window function +impl BuiltInWindowFunction { + pub fn return_type(&self, input_expr_types: &[DataType]) -> Result { + // Note that this function *must* return the same type that the respective physical expression returns + // or the execution panics. + + // Verify that this is a valid set of data types for this function + data_types(input_expr_types, &self.signature()) + // Original errors are all related to wrong function signature + // Aggregate them for better error message + .map_err(|_| { + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{self}"), + self.signature(), + input_expr_types, + ) + ) + })?; + + match self { + BuiltInWindowFunction::FirstValue + | BuiltInWindowFunction::LastValue + | BuiltInWindowFunction::NthValue => Ok(input_expr_types[0].clone()), + } + } + + /// The signatures supported by the built-in window function `fun`. + pub fn signature(&self) -> Signature { + // Note: The physical expression must accept the type returned by this function or the execution panics. + match self { + BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue => { + Signature::any(1, Volatility::Immutable) + } + BuiltInWindowFunction::NthValue => Signature::any(2, Volatility::Immutable), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use strum::IntoEnumIterator; + #[test] + // Test for BuiltInWindowFunction's Display and from_str() implementations. + // For each variant in BuiltInWindowFunction, it converts the variant to a string + // and then back to a variant. The test asserts that the original variant and + // the reconstructed variant are the same. This assertion is also necessary for + // function suggestion. See https://github.com/apache/datafusion/issues/8082 + fn test_display_and_from_str() { + for func_original in BuiltInWindowFunction::iter() { + let func_name = func_original.to_string(); + let func_from_str = BuiltInWindowFunction::from_str(&func_name).unwrap(); + assert_eq!(func_from_str, func_original); + } + } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 1b42c1ffa038..bdac69d07c65 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -27,7 +27,10 @@ use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; use crate::utils::expr_to_columns; use crate::Volatility; -use crate::{udaf, ExprSchemable, Operator, Signature, WindowFrame, WindowUDF}; +use crate::{ + udaf, BuiltInWindowFunction, ExprSchemable, Operator, Signature, WindowFrame, + WindowUDF, +}; use arrow::datatypes::{DataType, FieldRef}; use datafusion_common::cse::HashNode; @@ -698,6 +701,9 @@ impl AggregateFunction { #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] /// Defines which implementation of an aggregate function DataFusion should call. pub enum WindowFunctionDefinition { + /// A built in aggregate function that leverages an aggregate function + /// A a built-in window function + BuiltInWindowFunction(BuiltInWindowFunction), /// A user defined aggregate function AggregateUDF(Arc), /// A user defined aggregate function @@ -713,6 +719,9 @@ impl WindowFunctionDefinition { display_name: &str, ) -> Result { match self { + WindowFunctionDefinition::BuiltInWindowFunction(fun) => { + fun.return_type(input_expr_types) + } WindowFunctionDefinition::AggregateUDF(fun) => { fun.return_type(input_expr_types) } @@ -725,6 +734,7 @@ impl WindowFunctionDefinition { /// The signatures supported by the function `fun`. pub fn signature(&self) -> Signature { match self { + WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.signature(), WindowFunctionDefinition::AggregateUDF(fun) => fun.signature().clone(), WindowFunctionDefinition::WindowUDF(fun) => fun.signature().clone(), } @@ -733,6 +743,7 @@ impl WindowFunctionDefinition { /// Function's name for display pub fn name(&self) -> &str { match self { + WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.name(), WindowFunctionDefinition::WindowUDF(fun) => fun.name(), WindowFunctionDefinition::AggregateUDF(fun) => fun.name(), } @@ -742,12 +753,19 @@ impl WindowFunctionDefinition { impl Display for WindowFunctionDefinition { fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { + WindowFunctionDefinition::BuiltInWindowFunction(fun) => Display::fmt(fun, f), WindowFunctionDefinition::AggregateUDF(fun) => Display::fmt(fun, f), WindowFunctionDefinition::WindowUDF(fun) => Display::fmt(fun, f), } } } +impl From for WindowFunctionDefinition { + fn from(value: BuiltInWindowFunction) -> Self { + Self::BuiltInWindowFunction(value) + } +} + impl From> for WindowFunctionDefinition { fn from(value: Arc) -> Self { Self::AggregateUDF(value) @@ -773,11 +791,9 @@ impl From> for WindowFunctionDefinition { /// ``` /// # use datafusion_expr::{Expr, BuiltInWindowFunction, col, ExprFunctionExt}; /// # use datafusion_expr::expr::WindowFunction; -/// use datafusion_expr::test::function_stub::count_udaf; -/// use datafusion_expr::WindowFunctionDefinition::{AggregateUDF}; -/// // Create COUNT(a) OVER (PARTITION BY b ORDER BY c) +/// // Create FIRST_VALUE(a) OVER (PARTITION BY b ORDER BY c) /// let expr = Expr::WindowFunction( -/// WindowFunction::new(AggregateUDF(count_udaf()), vec![col("a")]) +/// WindowFunction::new(BuiltInWindowFunction::FirstValue, vec![col("a")]) /// ) /// .partition_by(vec![col("b")]) /// .order_by(vec![col("b").sort(true, true)]) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index d2c281c0077b..07a36672f272 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -478,6 +478,12 @@ impl Expr { .map(|e| e.get_type(schema)) .collect::>>()?; match fun { + WindowFunctionDefinition::BuiltInWindowFunction(window_fun) => { + let return_type = window_fun.return_type(&data_types)?; + let nullable = + !["RANK", "NTILE", "CUME_DIST"].contains(&window_fun.name()); + Ok((return_type, nullable)) + } WindowFunctionDefinition::AggregateUDF(udaf) => { let new_types = data_types_with_aggregate_udf(&data_types, udaf) .map_err(|err| { diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 3faa8192f3eb..701b2768531b 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -65,6 +65,7 @@ pub mod type_coercion; pub mod utils; pub mod var_provider; pub mod window_frame; +pub mod window_function; pub mod window_state; pub use built_in_window_function::BuiltInWindowFunction; diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs new file mode 100644 index 000000000000..be2b6575e2e9 --- /dev/null +++ b/datafusion/expr/src/window_function.rs @@ -0,0 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::{expr::WindowFunction, BuiltInWindowFunction, Expr, Literal}; + +/// Create an expression to represent the `nth_value` window function +pub fn nth_value(arg: Expr, n: i64) -> Expr { + Expr::WindowFunction(WindowFunction::new( + BuiltInWindowFunction::NthValue, + vec![arg, n.lit()], + )) +} diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index 2fcf82eeef25..a86714aaf93f 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -31,32 +31,51 @@ use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::window_doc_sections::DOC_SECTION_ANALYTICAL; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::{ - Documentation, PartitionEvaluator, ReversedUDWF, Signature, TypeSignature, + Documentation, Literal, PartitionEvaluator, ReversedUDWF, Signature, TypeSignature, Volatility, WindowUDFImpl, }; use datafusion_functions_window_common::field; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; use field::WindowUDFFieldArgs; -define_udwf_and_expr!( +get_or_init_udwf!( First, first_value, "returns the first value in the window frame", NthValue::first ); -define_udwf_and_expr!( +get_or_init_udwf!( Last, last_value, "returns the last value in the window frame", NthValue::last ); -define_udwf_and_expr!( +get_or_init_udwf!( NthValue, nth_value, "returns the nth value in the window frame", NthValue::nth ); +/// Create an expression to represent the `first_value` window function +/// +pub fn first_value(arg: datafusion_expr::Expr) -> datafusion_expr::Expr { + first_value_udwf().call(vec![arg]) +} + +/// Create an expression to represent the `last_value` window function +/// +pub fn last_value(arg: datafusion_expr::Expr) -> datafusion_expr::Expr { + last_value_udwf().call(vec![arg]) +} + +/// Create an expression to represent the `nth_value` window function +/// +pub fn nth_value(arg: datafusion_expr::Expr, n: Option) -> datafusion_expr::Expr { + let n_lit = n.map(|v| v.lit()).unwrap_or(ScalarValue::Null.lit()); + nth_value_udwf().call(vec![arg, n_lit]) +} + /// Tag to differentiate special use cases of the NTH_VALUE built-in window function. #[derive(Debug, Copy, Clone)] pub enum NthValueKind { diff --git a/datafusion/physical-expr/src/window/nth_value.rs b/datafusion/physical-expr/src/window/nth_value.rs new file mode 100644 index 000000000000..6ec3a23fc586 --- /dev/null +++ b/datafusion/physical-expr/src/window/nth_value.rs @@ -0,0 +1,415 @@ +// 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 for `FIRST_VALUE`, `LAST_VALUE`, and `NTH_VALUE` +//! functions that can be evaluated at run time during query execution. + +use std::any::Any; +use std::cmp::Ordering; +use std::ops::Range; +use std::sync::Arc; + +use crate::window::window_expr::{NthValueKind, NthValueState}; +use crate::window::BuiltInWindowFunctionExpr; +use crate::PhysicalExpr; + +use arrow::array::{Array, ArrayRef}; +use arrow::datatypes::{DataType, Field}; +use datafusion_common::Result; +use datafusion_common::ScalarValue; +use datafusion_expr::window_state::WindowAggState; +use datafusion_expr::PartitionEvaluator; + +/// nth_value expression +#[derive(Debug)] +pub struct NthValue { + name: String, + expr: Arc, + /// Output data type + data_type: DataType, + kind: NthValueKind, + ignore_nulls: bool, +} + +impl NthValue { + /// Create a new FIRST_VALUE window aggregate function + pub fn first( + name: impl Into, + expr: Arc, + data_type: DataType, + ignore_nulls: bool, + ) -> Self { + Self { + name: name.into(), + expr, + data_type, + kind: NthValueKind::First, + ignore_nulls, + } + } + + /// Create a new LAST_VALUE window aggregate function + pub fn last( + name: impl Into, + expr: Arc, + data_type: DataType, + ignore_nulls: bool, + ) -> Self { + Self { + name: name.into(), + expr, + data_type, + kind: NthValueKind::Last, + ignore_nulls, + } + } + + /// Create a new NTH_VALUE window aggregate function + pub fn nth( + name: impl Into, + expr: Arc, + data_type: DataType, + n: i64, + ignore_nulls: bool, + ) -> Result { + Ok(Self { + name: name.into(), + expr, + data_type, + kind: NthValueKind::Nth(n), + ignore_nulls, + }) + } + + /// Get the NTH_VALUE kind + pub fn get_kind(&self) -> NthValueKind { + self.kind + } +} + +impl BuiltInWindowFunctionExpr for NthValue { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result { + let nullable = true; + Ok(Field::new(&self.name, self.data_type.clone(), nullable)) + } + + fn expressions(&self) -> Vec> { + vec![Arc::clone(&self.expr)] + } + + fn name(&self) -> &str { + &self.name + } + + fn create_evaluator(&self) -> Result> { + let state = NthValueState { + finalized_result: None, + kind: self.kind, + }; + Ok(Box::new(NthValueEvaluator { + state, + ignore_nulls: self.ignore_nulls, + })) + } + + fn reverse_expr(&self) -> Option> { + let reversed_kind = match self.kind { + NthValueKind::First => NthValueKind::Last, + NthValueKind::Last => NthValueKind::First, + NthValueKind::Nth(idx) => NthValueKind::Nth(-idx), + }; + Some(Arc::new(Self { + name: self.name.clone(), + expr: Arc::clone(&self.expr), + data_type: self.data_type.clone(), + kind: reversed_kind, + ignore_nulls: self.ignore_nulls, + })) + } +} + +/// Value evaluator for nth_value functions +#[derive(Debug)] +pub(crate) struct NthValueEvaluator { + state: NthValueState, + ignore_nulls: bool, +} + +impl PartitionEvaluator for NthValueEvaluator { + /// When the window frame has a fixed beginning (e.g UNBOUNDED PRECEDING), + /// for some functions such as FIRST_VALUE, LAST_VALUE and NTH_VALUE, we + /// can memoize the result. Once result is calculated, it will always stay + /// same. Hence, we do not need to keep past data as we process the entire + /// dataset. + fn memoize(&mut self, state: &mut WindowAggState) -> Result<()> { + let out = &state.out_col; + let size = out.len(); + let mut buffer_size = 1; + // Decide if we arrived at a final result yet: + let (is_prunable, is_reverse_direction) = match self.state.kind { + NthValueKind::First => { + let n_range = + state.window_frame_range.end - state.window_frame_range.start; + (n_range > 0 && size > 0, false) + } + NthValueKind::Last => (true, true), + NthValueKind::Nth(n) => { + let n_range = + state.window_frame_range.end - state.window_frame_range.start; + match n.cmp(&0) { + Ordering::Greater => { + (n_range >= (n as usize) && size > (n as usize), false) + } + Ordering::Less => { + let reverse_index = (-n) as usize; + buffer_size = reverse_index; + // Negative index represents reverse direction. + (n_range >= reverse_index, true) + } + Ordering::Equal => (false, false), + } + } + }; + // Do not memoize results when nulls are ignored. + if is_prunable && !self.ignore_nulls { + if self.state.finalized_result.is_none() && !is_reverse_direction { + let result = ScalarValue::try_from_array(out, size - 1)?; + self.state.finalized_result = Some(result); + } + state.window_frame_range.start = + state.window_frame_range.end.saturating_sub(buffer_size); + } + Ok(()) + } + + fn evaluate( + &mut self, + values: &[ArrayRef], + range: &Range, + ) -> Result { + if let Some(ref result) = self.state.finalized_result { + Ok(result.clone()) + } else { + // FIRST_VALUE, LAST_VALUE, NTH_VALUE window functions take a single column, values will have size 1. + let arr = &values[0]; + let n_range = range.end - range.start; + if n_range == 0 { + // We produce None if the window is empty. + return ScalarValue::try_from(arr.data_type()); + } + + // Extract valid indices if ignoring nulls. + let valid_indices = if self.ignore_nulls { + // Calculate valid indices, inside the window frame boundaries + let slice = arr.slice(range.start, n_range); + let valid_indices = slice + .nulls() + .map(|nulls| { + nulls + .valid_indices() + // Add offset `range.start` to valid indices, to point correct index in the original arr. + .map(|idx| idx + range.start) + .collect::>() + }) + .unwrap_or_default(); + if valid_indices.is_empty() { + return ScalarValue::try_from(arr.data_type()); + } + Some(valid_indices) + } else { + None + }; + match self.state.kind { + NthValueKind::First => { + if let Some(valid_indices) = &valid_indices { + ScalarValue::try_from_array(arr, valid_indices[0]) + } else { + ScalarValue::try_from_array(arr, range.start) + } + } + NthValueKind::Last => { + if let Some(valid_indices) = &valid_indices { + ScalarValue::try_from_array( + arr, + valid_indices[valid_indices.len() - 1], + ) + } else { + ScalarValue::try_from_array(arr, range.end - 1) + } + } + NthValueKind::Nth(n) => { + match n.cmp(&0) { + Ordering::Greater => { + // SQL indices are not 0-based. + let index = (n as usize) - 1; + if index >= n_range { + // Outside the range, return NULL: + ScalarValue::try_from(arr.data_type()) + } else if let Some(valid_indices) = valid_indices { + if index >= valid_indices.len() { + return ScalarValue::try_from(arr.data_type()); + } + ScalarValue::try_from_array(&arr, valid_indices[index]) + } else { + ScalarValue::try_from_array(arr, range.start + index) + } + } + Ordering::Less => { + let reverse_index = (-n) as usize; + if n_range < reverse_index { + // Outside the range, return NULL: + ScalarValue::try_from(arr.data_type()) + } else if let Some(valid_indices) = valid_indices { + if reverse_index > valid_indices.len() { + return ScalarValue::try_from(arr.data_type()); + } + let new_index = + valid_indices[valid_indices.len() - reverse_index]; + ScalarValue::try_from_array(&arr, new_index) + } else { + ScalarValue::try_from_array( + arr, + range.start + n_range - reverse_index, + ) + } + } + Ordering::Equal => ScalarValue::try_from(arr.data_type()), + } + } + } + } + } + + fn supports_bounded_execution(&self) -> bool { + true + } + + fn uses_window_frame(&self) -> bool { + true + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::expressions::Column; + use arrow::{array::*, datatypes::*}; + use datafusion_common::cast::as_int32_array; + + fn test_i32_result(expr: NthValue, expected: Int32Array) -> Result<()> { + let arr: ArrayRef = Arc::new(Int32Array::from(vec![1, -2, 3, -4, 5, -6, 7, 8])); + let values = vec![arr]; + let schema = Schema::new(vec![Field::new("arr", DataType::Int32, false)]); + let batch = RecordBatch::try_new(Arc::new(schema), values.clone())?; + let mut ranges: Vec> = vec![]; + for i in 0..8 { + ranges.push(Range { + start: 0, + end: i + 1, + }) + } + let mut evaluator = expr.create_evaluator()?; + let values = expr.evaluate_args(&batch)?; + let result = ranges + .iter() + .map(|range| evaluator.evaluate(&values, range)) + .collect::>>()?; + let result = ScalarValue::iter_to_array(result.into_iter())?; + let result = as_int32_array(&result)?; + assert_eq!(expected, *result); + Ok(()) + } + + #[test] + fn first_value() -> Result<()> { + let first_value = NthValue::first( + "first_value".to_owned(), + Arc::new(Column::new("arr", 0)), + DataType::Int32, + false, + ); + test_i32_result(first_value, Int32Array::from(vec![1; 8]))?; + Ok(()) + } + + #[test] + fn last_value() -> Result<()> { + let last_value = NthValue::last( + "last_value".to_owned(), + Arc::new(Column::new("arr", 0)), + DataType::Int32, + false, + ); + test_i32_result( + last_value, + Int32Array::from(vec![ + Some(1), + Some(-2), + Some(3), + Some(-4), + Some(5), + Some(-6), + Some(7), + Some(8), + ]), + )?; + Ok(()) + } + + #[test] + fn nth_value_1() -> Result<()> { + let nth_value = NthValue::nth( + "nth_value".to_owned(), + Arc::new(Column::new("arr", 0)), + DataType::Int32, + 1, + false, + )?; + test_i32_result(nth_value, Int32Array::from(vec![1; 8]))?; + Ok(()) + } + + #[test] + fn nth_value_2() -> Result<()> { + let nth_value = NthValue::nth( + "nth_value".to_owned(), + Arc::new(Column::new("arr", 0)), + DataType::Int32, + 2, + false, + )?; + test_i32_result( + nth_value, + Int32Array::from(vec![ + None, + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + Some(-2), + ]), + )?; + Ok(()) + } +} 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 602efa54f8da..61c5e9584b8d 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1160,9 +1160,7 @@ mod tests { use std::task::{Context, Poll}; use std::time::Duration; - use crate::common::collect; use crate::expressions::PhysicalSortExpr; - use crate::memory::MemoryExec; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; use crate::windows::{create_window_expr, BoundedWindowAggExec, InputOrderMode}; @@ -1182,10 +1180,7 @@ mod tests { WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::expressions::{col, Column, NthValue}; - use datafusion_physical_expr::window::{ - BuiltInWindowExpr, BuiltInWindowFunctionExpr, - }; + use datafusion_physical_expr::expressions::{col, Column}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; use futures::future::Shared; @@ -1501,128 +1496,129 @@ mod tests { Ok(source) } - // Tests NTH_VALUE(negative index) with memoize feature. + // Tests NTH_VALUE(negative index) with memoize feature // To be able to trigger memoize feature for NTH_VALUE we need to // - feed BoundedWindowAggExec with batch stream data. // - Window frame should contain UNBOUNDED PRECEDING. // It hard to ensure these conditions are met, from the sql query. - #[tokio::test] - async fn test_window_nth_value_bounded_memoize() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); - - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - // Create a new batch of data to insert into the table - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], - )?; - - let memory_exec = MemoryExec::try_new( - &[vec![batch.clone(), batch.clone(), batch.clone()]], - Arc::clone(&schema), - None, - ) - .map(|e| Arc::new(e) as Arc)?; - let col_a = col("a", &schema)?; - let nth_value_func1 = NthValue::nth( - "nth_value(-1)", - Arc::clone(&col_a), - DataType::Int32, - 1, - false, - )? - .reverse_expr() - .unwrap(); - let nth_value_func2 = NthValue::nth( - "nth_value(-2)", - Arc::clone(&col_a), - DataType::Int32, - 2, - false, - )? - .reverse_expr() - .unwrap(); - let last_value_func = Arc::new(NthValue::last( - "last", - Arc::clone(&col_a), - DataType::Int32, - false, - )) as _; - let window_exprs = vec![ - // LAST_VALUE(a) - Arc::new(BuiltInWindowExpr::new( - last_value_func, - &[], - &LexOrdering::default(), - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - // NTH_VALUE(a, -1) - Arc::new(BuiltInWindowExpr::new( - nth_value_func1, - &[], - &LexOrdering::default(), - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - // NTH_VALUE(a, -2) - Arc::new(BuiltInWindowExpr::new( - nth_value_func2, - &[], - &LexOrdering::default(), - Arc::new(WindowFrame::new_bounds( - WindowFrameUnits::Rows, - WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - WindowFrameBound::CurrentRow, - )), - )) as _, - ]; - let physical_plan = BoundedWindowAggExec::try_new( - window_exprs, - memory_exec, - vec![], - InputOrderMode::Sorted, - ) - .map(|e| Arc::new(e) as Arc)?; - - let batches = collect(physical_plan.execute(0, task_ctx)?).await?; - - let expected = vec![ - "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[3]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+------+---------------+---------------+", - "| a | last | nth_value(-1) | nth_value(-2) |", - "+---+------+---------------+---------------+", - "| 1 | 1 | 1 | |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "| 1 | 1 | 1 | 3 |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "| 1 | 1 | 1 | 3 |", - "| 2 | 2 | 2 | 1 |", - "| 3 | 3 | 3 | 2 |", - "+---+------+---------------+---------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } + // #[tokio::test] + // async fn test_window_nth_value_bounded_memoize() -> Result<()> { + // let config = SessionConfig::new().with_target_partitions(1); + // let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); + // + // let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + // // Create a new batch of data to insert into the table + // let batch = RecordBatch::try_new( + // Arc::clone(&schema), + // vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], + // )?; + // + // let memory_exec = MemoryExec::try_new( + // &[vec![batch.clone(), batch.clone(), batch.clone()]], + // Arc::clone(&schema), + // None, + // ) + // .map(|e| Arc::new(e) as Arc)?; + // let col_a = col("a", &schema)?; + // // let nth_value_func1 = WindowFunctionDefinition::WindowUDF(nth_value_udwf()) + // // // NthValue::nth( + // // // "nth_value(-1)", + // // // Arc::clone(&col_a), + // // // DataType::Int32, + // // // 1, + // // // false, + // // // )? + // // // .reverse_expr() + // // .unwrap(); + // // let nth_value_func2 = NthValue::nth( + // // "nth_value(-2)", + // // Arc::clone(&col_a), + // // DataType::Int32, + // // 2, + // // false, + // // )? + // // .reverse_expr() + // // .unwrap(); + // // let last_value_func = Arc::new(NthValue::last( + // // "last", + // // Arc::clone(&col_a), + // // DataType::Int32, + // // false, + // // )) as _; + // // let window_exprs = vec![ + // // LAST_VALUE(a) + // // Arc::new(BuiltInWindowExpr::new( + // // last_value_func, + // // &[], + // // &LexOrdering::default(), + // // Arc::new(WindowFrame::new_bounds( + // // WindowFrameUnits::Rows, + // // WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + // // WindowFrameBound::CurrentRow, + // // )), + // // )) as _, + // // // NTH_VALUE(a, -1) + // // Arc::new(BuiltInWindowExpr::new( + // // nth_value_func1, + // // &[], + // // &LexOrdering::default(), + // // Arc::new(WindowFrame::new_bounds( + // // WindowFrameUnits::Rows, + // // WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + // // WindowFrameBound::CurrentRow, + // // )), + // // )) as _, + // // // NTH_VALUE(a, -2) + // // Arc::new(BuiltInWindowExpr::new( + // // nth_value_func2, + // // &[], + // // &LexOrdering::default(), + // // Arc::new(WindowFrame::new_bounds( + // // WindowFrameUnits::Rows, + // // WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + // // WindowFrameBound::CurrentRow, + // // )), + // // )) as _, + // // ]; + // let physical_plan = BoundedWindowAggExec::try_new( + // window_exprs, + // memory_exec, + // vec![], + // InputOrderMode::Sorted, + // ) + // .map(|e| Arc::new(e) as Arc)?; + // + // let batches = collect(physical_plan.execute(0, task_ctx)?).await?; + // + // let expected = vec![ + // "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", + // " MemoryExec: partitions=1, partition_sizes=[3]", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+------+---------------+---------------+", + // "| a | last | nth_value(-1) | nth_value(-2) |", + // "+---+------+---------------+---------------+", + // "| 1 | 1 | 1 | |", + // "| 2 | 2 | 2 | 1 |", + // "| 3 | 3 | 3 | 2 |", + // "| 1 | 1 | 1 | 3 |", + // "| 2 | 2 | 2 | 1 |", + // "| 3 | 3 | 3 | 2 |", + // "| 1 | 1 | 1 | 3 |", + // "| 2 | 2 | 2 | 1 |", + // "| 3 | 3 | 3 | 2 |", + // "+---+------+---------------+---------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } // This test, tests whether most recent row guarantee by the input batch of the `BoundedWindowAggExec` // helps `BoundedWindowAggExec` to generate low latency result in the `Linear` mode. diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1b2b4e70f920..aee0de94ea87 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -103,6 +103,9 @@ pub fn create_window_expr( ignore_nulls: bool, ) -> Result> { Ok(match fun { + WindowFunctionDefinition::BuiltInWindowFunction(_fun) => { + unreachable!() + } WindowFunctionDefinition::AggregateUDF(fun) => { let aggregate = AggregateExprBuilder::new(Arc::clone(fun), args.to_vec()) .schema(Arc::new(input_schema.clone())) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index eea125606719..37f33917ab5d 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -507,6 +507,17 @@ message ScalarUDFExprNode { enum BuiltInWindowFunction { UNSPECIFIED = 0; // https://protobuf.dev/programming-guides/dos-donts/#unspecified-enum + // ROW_NUMBER = 0; + // RANK = 1; + // DENSE_RANK = 2; + // PERCENT_RANK = 3; + // CUME_DIST = 4; + // NTILE = 5; + // LAG = 6; + // LEAD = 7; + // FIRST_VALUE = 8; + // LAST_VALUE = 9; + // NTH_VALUE = 10; } message WindowExprNode { diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 110905f3359c..4708e49d4565 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -149,8 +149,10 @@ impl From<&protobuf::StringifiedPlan> for StringifiedPlan { } impl From for BuiltInWindowFunction { - fn from(_built_in_function: protobuf::BuiltInWindowFunction) -> Self { - unreachable!() + fn from(built_in_function: protobuf::BuiltInWindowFunction) -> Self { + match built_in_function { + protobuf::BuiltInWindowFunction::Unspecified => todo!(), + } } } @@ -283,7 +285,25 @@ pub fn parse_expr( // TODO: support proto for null treatment match window_function { - window_expr_node::WindowFunction::BuiltInFunction(_) => unreachable!(), + window_expr_node::WindowFunction::BuiltInFunction(i) => { + let built_in_function = protobuf::BuiltInWindowFunction::try_from(*i) + .map_err(|_| Error::unknown("BuiltInWindowFunction", *i))? + .into(); + + let args = parse_exprs(&expr.exprs, registry, codec)?; + + Expr::WindowFunction(WindowFunction::new( + expr::WindowFunctionDefinition::BuiltInWindowFunction( + built_in_function, + ), + args, + )) + .partition_by(partition_by) + .order_by(order_by) + .window_frame(window_frame) + .build() + .map_err(Error::DataFusionError) + } window_expr_node::WindowFunction::Udaf(udaf_name) => { let udaf_function = match &expr.fun_definition { Some(buf) => codec.try_decode_udaf(udaf_name, buf)?, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index caceb3db164c..5ef64675280e 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -306,6 +306,7 @@ pub fn serialize_expr( null_treatment: _, }) => { let (window_function, fun_definition) = match fun { + WindowFunctionDefinition::BuiltInWindowFunction(_fun) => unreachable!(), WindowFunctionDefinition::AggregateUDF(aggr_udf) => { let mut buf = Vec::new(); let _ = codec.try_encode_udaf(aggr_udf, &mut buf); diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index b6543323efdf..31b59c2a9457 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -146,7 +146,15 @@ pub fn parse_physical_window_expr( let fun = if let Some(window_func) = proto.window_function.as_ref() { match window_func { - protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(_) => unreachable!(), + protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(n) => { + let f = protobuf::BuiltInWindowFunction::try_from(*n).map_err(|_| { + proto_error(format!( + "Received an unknown window builtin function: {n}" + )) + })?; + + WindowFunctionDefinition::BuiltInWindowFunction(f.into()) + } protobuf::physical_window_expr_node::WindowFunction::UserDefinedAggrFunction(udaf_name) => { WindowFunctionDefinition::AggregateUDF(match &proto.fun_definition { Some(buf) => codec.try_decode_udaf(udaf_name, buf)?, diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 991786212010..aab63dd8bd66 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -59,8 +59,7 @@ use datafusion::physical_plan::aggregates::{ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ - binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, NthValue, - PhysicalSortExpr, + binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, PhysicalSortExpr, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::DataSinkExec; @@ -74,9 +73,7 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; -use datafusion::physical_plan::windows::{ - BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, -}; +use datafusion::physical_plan::windows::{PlainAggregateWindowExpr, WindowAggExec}; use datafusion::physical_plan::{ExecutionPlan, Partitioning, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; @@ -272,32 +269,6 @@ fn roundtrip_window() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let window_frame = WindowFrame::new_bounds( - datafusion_expr::WindowFrameUnits::Range, - WindowFrameBound::Preceding(ScalarValue::Int64(None)), - WindowFrameBound::CurrentRow, - ); - - let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( - Arc::new(NthValue::first( - "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", - col("a", &schema)?, - DataType::Int64, - false, - )), - &[col("b", &schema)?], - &LexOrdering{ - inner: vec![PhysicalSortExpr { - expr: col("a", &schema)?, - options: SortOptions { - descending: false, - nulls_first: false, - }, - }] - }, - Arc::new(window_frame), - )); - let plain_aggr_window_expr = Arc::new(PlainAggregateWindowExpr::new( AggregateExprBuilder::new( avg_udaf(), @@ -335,11 +306,7 @@ fn roundtrip_window() -> Result<()> { let input = Arc::new(EmptyExec::new(schema.clone())); roundtrip_test(Arc::new(WindowAggExec::try_new( - vec![ - builtin_window_expr, - plain_aggr_window_expr, - sliding_aggr_window_expr, - ], + vec![plain_aggr_window_expr, sliding_aggr_window_expr], input, vec![col("b", &schema)?], )?)) From aa18c1d154812b2e8d9c9d776b10ac9b0e83ca1b Mon Sep 17 00:00:00 2001 From: buraksenb Date: Sun, 10 Nov 2024 23:43:48 +0300 Subject: [PATCH 13/18] continue --- datafusion/core/src/dataframe/mod.rs | 10 +- datafusion/expr/src/lib.rs | 1 - datafusion/expr/src/window_function.rs | 26 -- datafusion/functions-window/src/lib.rs | 2 - datafusion/functions-window/src/nth_value.rs | 8 +- .../physical-expr/src/window/nth_value.rs | 415 ------------------ datafusion/physical-plan/Cargo.toml | 1 + .../src/windows/bounded_window_agg_exec.rs | 241 +++++----- .../tests/cases/roundtrip_physical_plan.rs | 29 ++ datafusion/sqllogictest/test_files/errors.slt | 15 + datafusion/sqllogictest/test_files/window.slt | 2 +- 11 files changed, 178 insertions(+), 572 deletions(-) delete mode 100644 datafusion/expr/src/window_function.rs delete mode 100644 datafusion/physical-expr/src/window/nth_value.rs diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2c71cb80d755..bcf803573cdf 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1946,12 +1946,12 @@ mod tests { use datafusion_common_runtime::SpawnedTask; use datafusion_expr::expr::WindowFunction; use datafusion_expr::{ - cast, create_udf, lit, BuiltInWindowFunction, ExprFunctionExt, - ScalarFunctionImplementation, Volatility, WindowFrame, WindowFrameBound, - WindowFrameUnits, WindowFunctionDefinition, + cast, create_udf, lit, ExprFunctionExt, ScalarFunctionImplementation, Volatility, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::expr_fn::{array_agg, count_distinct}; use datafusion_functions_window::expr_fn::row_number; + use datafusion_functions_window::nth_value::first_value_udwf; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; use sqlparser::ast::NullTreatment; @@ -2177,9 +2177,7 @@ mod tests { // build plan using Table API let t = test_table().await?; let first_row = Expr::WindowFunction(WindowFunction::new( - WindowFunctionDefinition::BuiltInWindowFunction( - BuiltInWindowFunction::FirstValue, - ), + WindowFunctionDefinition::WindowUDF(first_value_udwf()), vec![col("aggregate_test_100.c1")], )) .partition_by(vec![col("aggregate_test_100.c2")]) diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 701b2768531b..3faa8192f3eb 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -65,7 +65,6 @@ pub mod type_coercion; pub mod utils; pub mod var_provider; pub mod window_frame; -pub mod window_function; pub mod window_state; pub use built_in_window_function::BuiltInWindowFunction; diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs deleted file mode 100644 index be2b6575e2e9..000000000000 --- a/datafusion/expr/src/window_function.rs +++ /dev/null @@ -1,26 +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. - -use crate::{expr::WindowFunction, BuiltInWindowFunction, Expr, Literal}; - -/// Create an expression to represent the `nth_value` window function -pub fn nth_value(arg: Expr, n: i64) -> Expr { - Expr::WindowFunction(WindowFunction::new( - BuiltInWindowFunction::NthValue, - vec![arg, n.lit()], - )) -} diff --git a/datafusion/functions-window/src/lib.rs b/datafusion/functions-window/src/lib.rs index de6e25bd454f..9f8e54a0423b 100644 --- a/datafusion/functions-window/src/lib.rs +++ b/datafusion/functions-window/src/lib.rs @@ -23,8 +23,6 @@ //! [DataFusion]: https://crates.io/crates/datafusion //! -extern crate core; - use std::sync::Arc; use log::debug; diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index a86714aaf93f..7f3d1cb07bca 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -27,7 +27,7 @@ use std::sync::OnceLock; use datafusion_common::arrow::array::ArrayRef; use datafusion_common::arrow::datatypes::{DataType, Field}; -use datafusion_common::{exec_err, Result, ScalarValue}; +use datafusion_common::{exec_datafusion_err, exec_err, Result, ScalarValue}; use datafusion_expr::window_doc_sections::DOC_SECTION_ANALYTICAL; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::{ @@ -215,7 +215,11 @@ impl WindowUDFImpl for NthValue { } let n = - match get_scalar_value_from_args(partition_evaluator_args.input_exprs(), 1)? + match get_scalar_value_from_args(partition_evaluator_args.input_exprs(), 1) + .map_err(|_e| { + exec_datafusion_err!( + "Expected a signed integer literal for the second argument of nth_value") + })? .map(get_signed_integer) { Some(Ok(n)) => { diff --git a/datafusion/physical-expr/src/window/nth_value.rs b/datafusion/physical-expr/src/window/nth_value.rs deleted file mode 100644 index 6ec3a23fc586..000000000000 --- a/datafusion/physical-expr/src/window/nth_value.rs +++ /dev/null @@ -1,415 +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 for `FIRST_VALUE`, `LAST_VALUE`, and `NTH_VALUE` -//! functions that can be evaluated at run time during query execution. - -use std::any::Any; -use std::cmp::Ordering; -use std::ops::Range; -use std::sync::Arc; - -use crate::window::window_expr::{NthValueKind, NthValueState}; -use crate::window::BuiltInWindowFunctionExpr; -use crate::PhysicalExpr; - -use arrow::array::{Array, ArrayRef}; -use arrow::datatypes::{DataType, Field}; -use datafusion_common::Result; -use datafusion_common::ScalarValue; -use datafusion_expr::window_state::WindowAggState; -use datafusion_expr::PartitionEvaluator; - -/// nth_value expression -#[derive(Debug)] -pub struct NthValue { - name: String, - expr: Arc, - /// Output data type - data_type: DataType, - kind: NthValueKind, - ignore_nulls: bool, -} - -impl NthValue { - /// Create a new FIRST_VALUE window aggregate function - pub fn first( - name: impl Into, - expr: Arc, - data_type: DataType, - ignore_nulls: bool, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - kind: NthValueKind::First, - ignore_nulls, - } - } - - /// Create a new LAST_VALUE window aggregate function - pub fn last( - name: impl Into, - expr: Arc, - data_type: DataType, - ignore_nulls: bool, - ) -> Self { - Self { - name: name.into(), - expr, - data_type, - kind: NthValueKind::Last, - ignore_nulls, - } - } - - /// Create a new NTH_VALUE window aggregate function - pub fn nth( - name: impl Into, - expr: Arc, - data_type: DataType, - n: i64, - ignore_nulls: bool, - ) -> Result { - Ok(Self { - name: name.into(), - expr, - data_type, - kind: NthValueKind::Nth(n), - ignore_nulls, - }) - } - - /// Get the NTH_VALUE kind - pub fn get_kind(&self) -> NthValueKind { - self.kind - } -} - -impl BuiltInWindowFunctionExpr for NthValue { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn field(&self) -> Result { - let nullable = true; - Ok(Field::new(&self.name, self.data_type.clone(), nullable)) - } - - fn expressions(&self) -> Vec> { - vec![Arc::clone(&self.expr)] - } - - fn name(&self) -> &str { - &self.name - } - - fn create_evaluator(&self) -> Result> { - let state = NthValueState { - finalized_result: None, - kind: self.kind, - }; - Ok(Box::new(NthValueEvaluator { - state, - ignore_nulls: self.ignore_nulls, - })) - } - - fn reverse_expr(&self) -> Option> { - let reversed_kind = match self.kind { - NthValueKind::First => NthValueKind::Last, - NthValueKind::Last => NthValueKind::First, - NthValueKind::Nth(idx) => NthValueKind::Nth(-idx), - }; - Some(Arc::new(Self { - name: self.name.clone(), - expr: Arc::clone(&self.expr), - data_type: self.data_type.clone(), - kind: reversed_kind, - ignore_nulls: self.ignore_nulls, - })) - } -} - -/// Value evaluator for nth_value functions -#[derive(Debug)] -pub(crate) struct NthValueEvaluator { - state: NthValueState, - ignore_nulls: bool, -} - -impl PartitionEvaluator for NthValueEvaluator { - /// When the window frame has a fixed beginning (e.g UNBOUNDED PRECEDING), - /// for some functions such as FIRST_VALUE, LAST_VALUE and NTH_VALUE, we - /// can memoize the result. Once result is calculated, it will always stay - /// same. Hence, we do not need to keep past data as we process the entire - /// dataset. - fn memoize(&mut self, state: &mut WindowAggState) -> Result<()> { - let out = &state.out_col; - let size = out.len(); - let mut buffer_size = 1; - // Decide if we arrived at a final result yet: - let (is_prunable, is_reverse_direction) = match self.state.kind { - NthValueKind::First => { - let n_range = - state.window_frame_range.end - state.window_frame_range.start; - (n_range > 0 && size > 0, false) - } - NthValueKind::Last => (true, true), - NthValueKind::Nth(n) => { - let n_range = - state.window_frame_range.end - state.window_frame_range.start; - match n.cmp(&0) { - Ordering::Greater => { - (n_range >= (n as usize) && size > (n as usize), false) - } - Ordering::Less => { - let reverse_index = (-n) as usize; - buffer_size = reverse_index; - // Negative index represents reverse direction. - (n_range >= reverse_index, true) - } - Ordering::Equal => (false, false), - } - } - }; - // Do not memoize results when nulls are ignored. - if is_prunable && !self.ignore_nulls { - if self.state.finalized_result.is_none() && !is_reverse_direction { - let result = ScalarValue::try_from_array(out, size - 1)?; - self.state.finalized_result = Some(result); - } - state.window_frame_range.start = - state.window_frame_range.end.saturating_sub(buffer_size); - } - Ok(()) - } - - fn evaluate( - &mut self, - values: &[ArrayRef], - range: &Range, - ) -> Result { - if let Some(ref result) = self.state.finalized_result { - Ok(result.clone()) - } else { - // FIRST_VALUE, LAST_VALUE, NTH_VALUE window functions take a single column, values will have size 1. - let arr = &values[0]; - let n_range = range.end - range.start; - if n_range == 0 { - // We produce None if the window is empty. - return ScalarValue::try_from(arr.data_type()); - } - - // Extract valid indices if ignoring nulls. - let valid_indices = if self.ignore_nulls { - // Calculate valid indices, inside the window frame boundaries - let slice = arr.slice(range.start, n_range); - let valid_indices = slice - .nulls() - .map(|nulls| { - nulls - .valid_indices() - // Add offset `range.start` to valid indices, to point correct index in the original arr. - .map(|idx| idx + range.start) - .collect::>() - }) - .unwrap_or_default(); - if valid_indices.is_empty() { - return ScalarValue::try_from(arr.data_type()); - } - Some(valid_indices) - } else { - None - }; - match self.state.kind { - NthValueKind::First => { - if let Some(valid_indices) = &valid_indices { - ScalarValue::try_from_array(arr, valid_indices[0]) - } else { - ScalarValue::try_from_array(arr, range.start) - } - } - NthValueKind::Last => { - if let Some(valid_indices) = &valid_indices { - ScalarValue::try_from_array( - arr, - valid_indices[valid_indices.len() - 1], - ) - } else { - ScalarValue::try_from_array(arr, range.end - 1) - } - } - NthValueKind::Nth(n) => { - match n.cmp(&0) { - Ordering::Greater => { - // SQL indices are not 0-based. - let index = (n as usize) - 1; - if index >= n_range { - // Outside the range, return NULL: - ScalarValue::try_from(arr.data_type()) - } else if let Some(valid_indices) = valid_indices { - if index >= valid_indices.len() { - return ScalarValue::try_from(arr.data_type()); - } - ScalarValue::try_from_array(&arr, valid_indices[index]) - } else { - ScalarValue::try_from_array(arr, range.start + index) - } - } - Ordering::Less => { - let reverse_index = (-n) as usize; - if n_range < reverse_index { - // Outside the range, return NULL: - ScalarValue::try_from(arr.data_type()) - } else if let Some(valid_indices) = valid_indices { - if reverse_index > valid_indices.len() { - return ScalarValue::try_from(arr.data_type()); - } - let new_index = - valid_indices[valid_indices.len() - reverse_index]; - ScalarValue::try_from_array(&arr, new_index) - } else { - ScalarValue::try_from_array( - arr, - range.start + n_range - reverse_index, - ) - } - } - Ordering::Equal => ScalarValue::try_from(arr.data_type()), - } - } - } - } - } - - fn supports_bounded_execution(&self) -> bool { - true - } - - fn uses_window_frame(&self) -> bool { - true - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::Column; - use arrow::{array::*, datatypes::*}; - use datafusion_common::cast::as_int32_array; - - fn test_i32_result(expr: NthValue, expected: Int32Array) -> Result<()> { - let arr: ArrayRef = Arc::new(Int32Array::from(vec![1, -2, 3, -4, 5, -6, 7, 8])); - let values = vec![arr]; - let schema = Schema::new(vec![Field::new("arr", DataType::Int32, false)]); - let batch = RecordBatch::try_new(Arc::new(schema), values.clone())?; - let mut ranges: Vec> = vec![]; - for i in 0..8 { - ranges.push(Range { - start: 0, - end: i + 1, - }) - } - let mut evaluator = expr.create_evaluator()?; - let values = expr.evaluate_args(&batch)?; - let result = ranges - .iter() - .map(|range| evaluator.evaluate(&values, range)) - .collect::>>()?; - let result = ScalarValue::iter_to_array(result.into_iter())?; - let result = as_int32_array(&result)?; - assert_eq!(expected, *result); - Ok(()) - } - - #[test] - fn first_value() -> Result<()> { - let first_value = NthValue::first( - "first_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - false, - ); - test_i32_result(first_value, Int32Array::from(vec![1; 8]))?; - Ok(()) - } - - #[test] - fn last_value() -> Result<()> { - let last_value = NthValue::last( - "last_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - false, - ); - test_i32_result( - last_value, - Int32Array::from(vec![ - Some(1), - Some(-2), - Some(3), - Some(-4), - Some(5), - Some(-6), - Some(7), - Some(8), - ]), - )?; - Ok(()) - } - - #[test] - fn nth_value_1() -> Result<()> { - let nth_value = NthValue::nth( - "nth_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - 1, - false, - )?; - test_i32_result(nth_value, Int32Array::from(vec![1; 8]))?; - Ok(()) - } - - #[test] - fn nth_value_2() -> Result<()> { - let nth_value = NthValue::nth( - "nth_value".to_owned(), - Arc::new(Column::new("arr", 0)), - DataType::Int32, - 2, - false, - )?; - test_i32_result( - nth_value, - Int32Array::from(vec![ - None, - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - Some(-2), - ]), - )?; - Ok(()) - } -} diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index a9f9b22fafda..64fd0f49a233 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -70,6 +70,7 @@ tokio = { workspace = true } [dev-dependencies] criterion = { version = "0.5", features = ["async_futures"] } datafusion-functions-aggregate = { workspace = true } +datafusion-functions-window = { workspace = true } rstest = { workspace = true } rstest_reuse = "0.7.0" tokio = { workspace = true, features = [ 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 61c5e9584b8d..0a898bd852bb 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1180,9 +1180,13 @@ mod tests { WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::expressions::{col, Column}; + use datafusion_functions_window::nth_value::first_value_udwf; + use datafusion_functions_window::nth_value::last_value_udwf; + use datafusion_functions_window::nth_value::nth_value_udwf; + use datafusion_physical_expr::expressions::{col, lit, Column}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; + use crate::memory::MemoryExec; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1501,124 +1505,123 @@ mod tests { // - feed BoundedWindowAggExec with batch stream data. // - Window frame should contain UNBOUNDED PRECEDING. // It hard to ensure these conditions are met, from the sql query. - // #[tokio::test] - // async fn test_window_nth_value_bounded_memoize() -> Result<()> { - // let config = SessionConfig::new().with_target_partitions(1); - // let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); - // - // let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - // // Create a new batch of data to insert into the table - // let batch = RecordBatch::try_new( - // Arc::clone(&schema), - // vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], - // )?; - // - // let memory_exec = MemoryExec::try_new( - // &[vec![batch.clone(), batch.clone(), batch.clone()]], - // Arc::clone(&schema), - // None, - // ) - // .map(|e| Arc::new(e) as Arc)?; - // let col_a = col("a", &schema)?; - // // let nth_value_func1 = WindowFunctionDefinition::WindowUDF(nth_value_udwf()) - // // // NthValue::nth( - // // // "nth_value(-1)", - // // // Arc::clone(&col_a), - // // // DataType::Int32, - // // // 1, - // // // false, - // // // )? - // // // .reverse_expr() - // // .unwrap(); - // // let nth_value_func2 = NthValue::nth( - // // "nth_value(-2)", - // // Arc::clone(&col_a), - // // DataType::Int32, - // // 2, - // // false, - // // )? - // // .reverse_expr() - // // .unwrap(); - // // let last_value_func = Arc::new(NthValue::last( - // // "last", - // // Arc::clone(&col_a), - // // DataType::Int32, - // // false, - // // )) as _; - // // let window_exprs = vec![ - // // LAST_VALUE(a) - // // Arc::new(BuiltInWindowExpr::new( - // // last_value_func, - // // &[], - // // &LexOrdering::default(), - // // Arc::new(WindowFrame::new_bounds( - // // WindowFrameUnits::Rows, - // // WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - // // WindowFrameBound::CurrentRow, - // // )), - // // )) as _, - // // // NTH_VALUE(a, -1) - // // Arc::new(BuiltInWindowExpr::new( - // // nth_value_func1, - // // &[], - // // &LexOrdering::default(), - // // Arc::new(WindowFrame::new_bounds( - // // WindowFrameUnits::Rows, - // // WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - // // WindowFrameBound::CurrentRow, - // // )), - // // )) as _, - // // // NTH_VALUE(a, -2) - // // Arc::new(BuiltInWindowExpr::new( - // // nth_value_func2, - // // &[], - // // &LexOrdering::default(), - // // Arc::new(WindowFrame::new_bounds( - // // WindowFrameUnits::Rows, - // // WindowFrameBound::Preceding(ScalarValue::UInt64(None)), - // // WindowFrameBound::CurrentRow, - // // )), - // // )) as _, - // // ]; - // let physical_plan = BoundedWindowAggExec::try_new( - // window_exprs, - // memory_exec, - // vec![], - // InputOrderMode::Sorted, - // ) - // .map(|e| Arc::new(e) as Arc)?; - // - // let batches = collect(physical_plan.execute(0, task_ctx)?).await?; - // - // let expected = vec![ - // "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", - // " MemoryExec: partitions=1, partition_sizes=[3]", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+------+---------------+---------------+", - // "| a | last | nth_value(-1) | nth_value(-2) |", - // "+---+------+---------------+---------------+", - // "| 1 | 1 | 1 | |", - // "| 2 | 2 | 2 | 1 |", - // "| 3 | 3 | 3 | 2 |", - // "| 1 | 1 | 1 | 3 |", - // "| 2 | 2 | 2 | 1 |", - // "| 3 | 3 | 3 | 2 |", - // "| 1 | 1 | 1 | 3 |", - // "| 2 | 2 | 2 | 1 |", - // "| 3 | 3 | 3 | 2 |", - // "+---+------+---------------+---------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } + #[tokio::test] + async fn test_window_nth_value_bounded_memoize() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); + + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + // Create a new batch of data to insert into the table + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], + )?; + + let memory_exec = MemoryExec::try_new( + &[vec![batch.clone(), batch.clone(), batch.clone()]], + Arc::clone(&schema), + None, + ) + .map(|e| Arc::new(e) as Arc)?; + let col_a = col("a", &schema)?; + let nth_value_func1 = NthValue::nth( + "nth_value(-1)", + Arc::clone(&col_a), + DataType::Int32, + 1, + false, + )? + .reverse_expr() + .unwrap(); + let nth_value_func2 = NthValue::nth( + "nth_value(-2)", + Arc::clone(&col_a), + DataType::Int32, + 2, + false, + )? + .reverse_expr() + .unwrap(); + let last_value_func = Arc::new(NthValue::last( + "last", + Arc::clone(&col_a), + DataType::Int32, + false, + )) as _; + let window_exprs = vec![ + // LAST_VALUE(a) + Arc::new(BuiltInWindowExpr::new( + last_value_func, + &[], + &LexOrdering::default(), + Arc::new(WindowFrame::new_bounds( + WindowFrameUnits::Rows, + WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + WindowFrameBound::CurrentRow, + )), + )) as _, + // NTH_VALUE(a, -1) + Arc::new(BuiltInWindowExpr::new( + nth_value_func1, + &[], + &LexOrdering::default(), + Arc::new(WindowFrame::new_bounds( + WindowFrameUnits::Rows, + WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + WindowFrameBound::CurrentRow, + )), + )) as _, + // NTH_VALUE(a, -2) + Arc::new(BuiltInWindowExpr::new( + nth_value_func2, + &[], + &LexOrdering::default(), + Arc::new(WindowFrame::new_bounds( + WindowFrameUnits::Rows, + WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + WindowFrameBound::CurrentRow, + )), + )) as _, + ]; + let physical_plan = BoundedWindowAggExec::try_new( + window_exprs, + memory_exec, + vec![], + InputOrderMode::Sorted, + ) + .map(|e| Arc::new(e) as Arc)?; + + let batches = collect(physical_plan.execute(0, task_ctx)?).await?; + + let expected = vec![ + "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", + " MemoryExec: partitions=1, partition_sizes=[3]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+------+---------------+---------------+", + "| a | last | nth_value(-1) | nth_value(-2) |", + "+---+------+---------------+---------------+", + "| 1 | 1 | 1 | |", + "| 2 | 2 | 2 | 1 |", + "| 3 | 3 | 3 | 2 |", + "| 1 | 1 | 1 | 3 |", + "| 2 | 2 | 2 | 1 |", + "| 3 | 3 | 3 | 2 |", + "| 1 | 1 | 1 | 3 |", + "| 2 | 2 | 2 | 1 |", + "| 3 | 3 | 3 | 2 |", + "+---+------+---------------+---------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } // This test, tests whether most recent row guarantee by the input batch of the `BoundedWindowAggExec` // helps `BoundedWindowAggExec` to generate low latency result in the `Linear` mode. diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index aab63dd8bd66..88939b5bccf4 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -269,6 +269,35 @@ fn roundtrip_window() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); + let _window_frame = WindowFrame::new_bounds( + datafusion_expr::WindowFrameUnits::Range, + WindowFrameBound::Preceding(ScalarValue::Int64(None)), + WindowFrameBound::CurrentRow, + ); + + // let udwf_window_expr = Expr::WindowFunction(WindowFunction::new( + // WindowFunctionDefinition::WindowUDF(first_value_udwf()), + + // let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( + // Arc::new(NthValue::first( + // "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", + // col("a", &schema)?, + // DataType::Int64, + // false, + // )), + // &[col("b", &schema)?], + // &LexOrdering{ + // inner: vec![PhysicalSortExpr { + // expr: col("a", &schema)?, + // options: SortOptions { + // descending: false, + // nulls_first: false, + // }, + // }] + // }, + // Arc::new(window_frame), + // )); + let plain_aggr_window_expr = Arc::new(PlainAggregateWindowExpr::new( AggregateExprBuilder::new( avg_udaf(), diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index db180d2cf14d..4911a18dc019 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -136,6 +136,21 @@ from aggregate_test_100 order by c9 +statement error DataFusion error: Error during planning: Invalid function 'nth_vlue'.\nDid you mean 'nth_value'? +SELECT + NTH_VLUE(c4, 2) OVER() + FROM aggregate_test_100 + ORDER BY c9 + LIMIT 5; + +statement error DataFusion error: Error during planning: Invalid function 'frst_value'.\nDid you mean 'first_value'? +SELECT + FRST_VALUE(c4, 2) OVER() + FROM aggregate_test_100 + ORDER BY c9 + LIMIT 5; + + query error DataFusion error: Arrow error: Cast error: Cannot cast string 'foo' to value of Int64 type create table foo as values (1), ('foo'); diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 5bbe5cfc172a..8e3559a32684 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4892,7 +4892,7 @@ DROP TABLE t1; statement ok CREATE TABLE t1(v1 BIGINT); -query error DataFusion error: This feature is not implemented: There is only support Literal types for field at idx: 1 in Window Function +query error DataFusion error: Execution error: Expected a signed integer literal for the second argument of nth_value SELECT NTH_VALUE('+Inf'::Double, v1) OVER (PARTITION BY v1) FROM t1; statement ok From be02f030e71708e979dc9f80df046836622c3562 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 11 Nov 2024 10:29:04 +0300 Subject: [PATCH 14/18] Update roundtrip_physical_plan.rs --- .../tests/cases/roundtrip_physical_plan.rs | 53 ++++++++++--------- 1 file changed, 28 insertions(+), 25 deletions(-) diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 88939b5bccf4..5cea26356e4b 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,7 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; use arrow::datatypes::{Fields, TimeUnit}; -use datafusion::physical_expr::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_expr::dml::InsertOp; use datafusion_functions_aggregate::approx_percentile_cont::approx_percentile_cont_udaf; @@ -269,34 +269,33 @@ fn roundtrip_window() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let _window_frame = WindowFrame::new_bounds( + let window_frame = WindowFrame::new_bounds( datafusion_expr::WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::Int64(None)), WindowFrameBound::CurrentRow, ); - // let udwf_window_expr = Expr::WindowFunction(WindowFunction::new( - // WindowFunctionDefinition::WindowUDF(first_value_udwf()), - - // let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( - // Arc::new(NthValue::first( - // "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", - // col("a", &schema)?, - // DataType::Int64, - // false, - // )), - // &[col("b", &schema)?], - // &LexOrdering{ - // inner: vec![PhysicalSortExpr { - // expr: col("a", &schema)?, - // options: SortOptions { - // descending: false, - // nulls_first: false, - // }, - // }] - // }, - // Arc::new(window_frame), - // )); + let args = vec![cast(col("a", &schema)?, &schema, DataType::Int64)?]; + let nth_value_expr = AggregateExprBuilder::new(nth_value_udaf(), args) + .order_by(LexOrdering { + inner: vec![PhysicalSortExpr { + expr: col("a", &schema)?, + options: SortOptions { + descending: false, + nulls_first: false, + }, + }], + }) + .schema(Arc::clone(&schema)) + .alias("FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") + .build() + .map(Arc::new)?; + let sliding_aggr_window_nth_value = Arc::new(SlidingAggregateWindowExpr::new( + nth_value_expr, + &[col("b", &schema)?], + &LexOrdering::default(), + Arc::new(window_frame), + )); let plain_aggr_window_expr = Arc::new(PlainAggregateWindowExpr::new( AggregateExprBuilder::new( @@ -335,7 +334,11 @@ fn roundtrip_window() -> Result<()> { let input = Arc::new(EmptyExec::new(schema.clone())); roundtrip_test(Arc::new(WindowAggExec::try_new( - vec![plain_aggr_window_expr, sliding_aggr_window_expr], + vec![ + plain_aggr_window_expr, + sliding_aggr_window_expr, + sliding_aggr_window_nth_value, + ], input, vec![col("b", &schema)?], )?)) From 5c4e0f6ec67945b0e6f70662b5fd75513f77af65 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 11 Nov 2024 10:57:43 +0300 Subject: [PATCH 15/18] udwf, not udaf --- datafusion/physical-plan/src/windows/mod.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 31 ++++++++++--------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index aee0de94ea87..d2eb14638c71 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -158,7 +158,7 @@ fn window_expr_from_aggregate_expr( } /// Creates a `BuiltInWindowFunctionExpr` suitable for a user defined window function -fn create_udwf_window_expr( +pub fn create_udwf_window_expr( fun: &Arc, args: &[Arc], input_schema: &Schema, diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 5cea26356e4b..345a2e81fb07 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -47,9 +47,10 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; +use datafusion::functions_window::nth_value::nth_value_udwf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; -use datafusion::physical_expr::window::SlidingAggregateWindowExpr; +use datafusion::physical_expr::window::{BuiltInWindowExpr, SlidingAggregateWindowExpr}; use datafusion::physical_expr::{ LexOrdering, LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, }; @@ -73,7 +74,9 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; -use datafusion::physical_plan::windows::{PlainAggregateWindowExpr, WindowAggExec}; +use datafusion::physical_plan::windows::{ + create_udwf_window_expr, PlainAggregateWindowExpr, WindowAggExec, +}; use datafusion::physical_plan::{ExecutionPlan, Partitioning, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; @@ -85,9 +88,11 @@ use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, not_impl_err, DataFusionError, Result, UnnestOptions, }; +use datafusion_expr::WindowFunctionDefinition::WindowUDF; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ColumnarValue, ScalarUDF, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, + WindowFunctionDefinition, }; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::nth_value::nth_value_udaf; @@ -96,6 +101,7 @@ use datafusion_proto::physical_plan::{ AsExecutionPlan, DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, }; use datafusion_proto::protobuf; +use datafusion_proto::protobuf::logical_expr_node::ExprType::WindowExpr; /// Perform a serde roundtrip and assert that the string representation of the before and after plans /// are identical. Note that this often isn't sufficient to guarantee that no information is @@ -275,9 +281,12 @@ fn roundtrip_window() -> Result<()> { WindowFrameBound::CurrentRow, ); - let args = vec![cast(col("a", &schema)?, &schema, DataType::Int64)?]; - let nth_value_expr = AggregateExprBuilder::new(nth_value_udaf(), args) - .order_by(LexOrdering { + let nth_value_window = + create_udwf_window_expr(&nth_value_udwf(), &[col("a", &schema)?], schema.as_ref(), "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW".to_string(), false)?; + let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( + nth_value_window, + &[col("b", &schema)?], + &LexOrdering { inner: vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -285,15 +294,7 @@ fn roundtrip_window() -> Result<()> { nulls_first: false, }, }], - }) - .schema(Arc::clone(&schema)) - .alias("FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") - .build() - .map(Arc::new)?; - let sliding_aggr_window_nth_value = Arc::new(SlidingAggregateWindowExpr::new( - nth_value_expr, - &[col("b", &schema)?], - &LexOrdering::default(), + }, Arc::new(window_frame), )); @@ -337,7 +338,7 @@ fn roundtrip_window() -> Result<()> { vec![ plain_aggr_window_expr, sliding_aggr_window_expr, - sliding_aggr_window_nth_value, + builtin_window_expr, ], input, vec![col("b", &schema)?], From 9bbeef4e09f6516493176384b269d6b223af2ecb Mon Sep 17 00:00:00 2001 From: buraksenb Date: Mon, 11 Nov 2024 12:31:41 +0300 Subject: [PATCH 16/18] fix bounded but not fixed roundtrip --- .../src/windows/bounded_window_agg_exec.rs | 48 ++++++++++++------- .../tests/cases/roundtrip_physical_plan.rs | 16 +++++-- 2 files changed, 41 insertions(+), 23 deletions(-) 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 0a898bd852bb..50666f6cc368 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1163,7 +1163,9 @@ mod tests { use crate::expressions::PhysicalSortExpr; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; - use crate::windows::{create_window_expr, BoundedWindowAggExec, InputOrderMode}; + use crate::windows::{ + create_udwf_window_expr, create_window_expr, BoundedWindowAggExec, InputOrderMode, + }; use crate::{execute_stream, get_plan_string, ExecutionPlan}; use arrow_array::builder::{Int64Builder, UInt64Builder}; @@ -1180,13 +1182,14 @@ mod tests { WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_functions_aggregate::count::count_udaf; - use datafusion_functions_window::nth_value::first_value_udwf; use datafusion_functions_window::nth_value::last_value_udwf; use datafusion_functions_window::nth_value::nth_value_udwf; - use datafusion_physical_expr::expressions::{col, lit, Column}; + use datafusion_physical_expr::expressions::{col, Column, Literal}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; + use crate::common::collect; use crate::memory::MemoryExec; + use datafusion_physical_expr::window::BuiltInWindowExpr; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1524,30 +1527,39 @@ mod tests { ) .map(|e| Arc::new(e) as Arc)?; let col_a = col("a", &schema)?; - let nth_value_func1 = NthValue::nth( - "nth_value(-1)", - Arc::clone(&col_a), - DataType::Int32, - 1, + let nth_value_func1 = create_udwf_window_expr( + &nth_value_udwf(), + &[ + Arc::clone(&col_a), + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + ], + &schema, + "nth_value(-1)".to_string(), false, )? .reverse_expr() .unwrap(); - let nth_value_func2 = NthValue::nth( - "nth_value(-2)", - Arc::clone(&col_a), - DataType::Int32, - 2, + let nth_value_func2 = create_udwf_window_expr( + &nth_value_udwf(), + &[ + Arc::clone(&col_a), + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))), + ], + &schema, + "nth_value(-2)".to_string(), false, )? .reverse_expr() .unwrap(); - let last_value_func = Arc::new(NthValue::last( - "last", - Arc::clone(&col_a), - DataType::Int32, + + let last_value_func = create_udwf_window_expr( + &last_value_udwf(), + &[Arc::clone(&col_a)], + &schema, + "last".to_string(), false, - )) as _; + )?; + let window_exprs = vec![ // LAST_VALUE(a) Arc::new(BuiltInWindowExpr::new( diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 345a2e81fb07..a0cc730237d3 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,7 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; use arrow::datatypes::{Fields, TimeUnit}; -use datafusion::physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_expr::dml::InsertOp; use datafusion_functions_aggregate::approx_percentile_cont::approx_percentile_cont_udaf; @@ -47,7 +47,7 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; -use datafusion::functions_window::nth_value::nth_value_udwf; +use datafusion::functions_window::nth_value::first_value_udwf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::{BuiltInWindowExpr, SlidingAggregateWindowExpr}; @@ -281,10 +281,16 @@ fn roundtrip_window() -> Result<()> { WindowFrameBound::CurrentRow, ); - let nth_value_window = - create_udwf_window_expr(&nth_value_udwf(), &[col("a", &schema)?], schema.as_ref(), "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW".to_string(), false)?; + let first_value_window = create_udwf_window_expr( + &first_value_udwf(), + &[col("a", &schema)?], + schema.as_ref(), + "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW".to_string(), + false, + )?; + // "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( - nth_value_window, + first_value_window, &[col("b", &schema)?], &LexOrdering { inner: vec![PhysicalSortExpr { From 9e7e5e2db8fb5b8cfc47b4ab8bf643b07052bdf5 Mon Sep 17 00:00:00 2001 From: buraksenb Date: Mon, 11 Nov 2024 16:38:36 +0300 Subject: [PATCH 17/18] added --- datafusion/functions-window/src/nth_value.rs | 5 +-- .../tests/cases/roundtrip_logical_plan.rs | 4 ++ .../tests/cases/roundtrip_physical_plan.rs | 45 ++----------------- datafusion/sqllogictest/test_files/errors.slt | 6 +-- 4 files changed, 12 insertions(+), 48 deletions(-) diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index 7f3d1cb07bca..cff49c1f4c85 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -71,9 +71,8 @@ pub fn last_value(arg: datafusion_expr::Expr) -> datafusion_expr::Expr { /// Create an expression to represent the `nth_value` window function /// -pub fn nth_value(arg: datafusion_expr::Expr, n: Option) -> datafusion_expr::Expr { - let n_lit = n.map(|v| v.lit()).unwrap_or(ScalarValue::Null.lit()); - nth_value_udwf().call(vec![arg, n_lit]) +pub fn nth_value(arg: datafusion_expr::Expr, n: i64) -> datafusion_expr::Expr { + nth_value_udwf().call(vec![arg, n.lit()]) } /// Tag to differentiate special use cases of the NTH_VALUE built-in window function. diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index ecfbaee23537..d2710d974491 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -48,6 +48,7 @@ use datafusion::functions_aggregate::expr_fn::{ }; use datafusion::functions_aggregate::min_max::max_udaf; use datafusion::functions_nested::map::map; +use datafusion::functions_window; use datafusion::functions_window::expr_fn::{ cume_dist, dense_rank, lag, lead, ntile, percent_rank, rank, row_number, }; @@ -911,6 +912,9 @@ async fn roundtrip_expr_api() -> Result<()> { count_distinct(lit(1)), first_value(lit(1), None), first_value(lit(1), Some(vec![lit(2).sort(true, true)])), + functions_window::nth_value::first_value(lit(1)), + functions_window::nth_value::last_value(lit(1)), + functions_window::nth_value::nth_value(lit(1), 1), avg(lit(1.5)), covar_samp(lit(1.5), lit(2.2)), covar_pop(lit(1.5), lit(2.2)), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index a0cc730237d3..aab63dd8bd66 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -47,10 +47,9 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; -use datafusion::functions_window::nth_value::first_value_udwf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; -use datafusion::physical_expr::window::{BuiltInWindowExpr, SlidingAggregateWindowExpr}; +use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{ LexOrdering, LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, }; @@ -74,9 +73,7 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; -use datafusion::physical_plan::windows::{ - create_udwf_window_expr, PlainAggregateWindowExpr, WindowAggExec, -}; +use datafusion::physical_plan::windows::{PlainAggregateWindowExpr, WindowAggExec}; use datafusion::physical_plan::{ExecutionPlan, Partitioning, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; @@ -88,11 +85,9 @@ use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, not_impl_err, DataFusionError, Result, UnnestOptions, }; -use datafusion_expr::WindowFunctionDefinition::WindowUDF; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ColumnarValue, ScalarUDF, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, - WindowFunctionDefinition, }; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::nth_value::nth_value_udaf; @@ -101,7 +96,6 @@ use datafusion_proto::physical_plan::{ AsExecutionPlan, DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, }; use datafusion_proto::protobuf; -use datafusion_proto::protobuf::logical_expr_node::ExprType::WindowExpr; /// Perform a serde roundtrip and assert that the string representation of the before and after plans /// are identical. Note that this often isn't sufficient to guarantee that no information is @@ -275,35 +269,6 @@ fn roundtrip_window() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let window_frame = WindowFrame::new_bounds( - datafusion_expr::WindowFrameUnits::Range, - WindowFrameBound::Preceding(ScalarValue::Int64(None)), - WindowFrameBound::CurrentRow, - ); - - let first_value_window = create_udwf_window_expr( - &first_value_udwf(), - &[col("a", &schema)?], - schema.as_ref(), - "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW".to_string(), - false, - )?; - // "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", - let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( - first_value_window, - &[col("b", &schema)?], - &LexOrdering { - inner: vec![PhysicalSortExpr { - expr: col("a", &schema)?, - options: SortOptions { - descending: false, - nulls_first: false, - }, - }], - }, - Arc::new(window_frame), - )); - let plain_aggr_window_expr = Arc::new(PlainAggregateWindowExpr::new( AggregateExprBuilder::new( avg_udaf(), @@ -341,11 +306,7 @@ fn roundtrip_window() -> Result<()> { let input = Arc::new(EmptyExec::new(schema.clone())); roundtrip_test(Arc::new(WindowAggExec::try_new( - vec![ - plain_aggr_window_expr, - sliding_aggr_window_expr, - builtin_window_expr, - ], + vec![plain_aggr_window_expr, sliding_aggr_window_expr], input, vec![col("b", &schema)?], )?)) diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index 4911a18dc019..dcb0e59da848 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -119,7 +119,7 @@ regr_slope(c11, '2') over () as min1 from aggregate_test_100 order by c9 -# WindowFunction with BuiltInWindowFunction wrong signature +# WindowFunction wrong signature statement error select c9, @@ -132,10 +132,9 @@ DataFusion error: Error during planning: Error during planning: Coercion from [I nth_value() nth_value(Any) nth_value(Any, Any) -from aggregate_test_100 -order by c9 +# nth_value with wrong name statement error DataFusion error: Error during planning: Invalid function 'nth_vlue'.\nDid you mean 'nth_value'? SELECT NTH_VLUE(c4, 2) OVER() @@ -143,6 +142,7 @@ SELECT ORDER BY c9 LIMIT 5; +# first_value with wrong name statement error DataFusion error: Error during planning: Invalid function 'frst_value'.\nDid you mean 'first_value'? SELECT FRST_VALUE(c4, 2) OVER() From 073d8e8167c4a5d974340224c9291cc2af0dc477 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Mon, 11 Nov 2024 17:07:16 +0300 Subject: [PATCH 18/18] Update datafusion/sqllogictest/test_files/errors.slt Co-authored-by: Sherin Jacob --- datafusion/sqllogictest/test_files/errors.slt | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index dcb0e59da848..0dcac97a23aa 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -120,18 +120,12 @@ from aggregate_test_100 order by c9 # WindowFunction wrong signature -statement error +statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Int32, Int64, Int64\] to the signature OneOf\(\[Any\(0\), Any\(1\), Any\(2\)\]\) failed select c9, nth_value(c5, 2, 3) over (order by c9) as nv1 from aggregate_test_100 order by c9 ----- -DataFusion error: Error during planning: Error during planning: Coercion from [Int32, Int64, Int64] to the signature OneOf([Any(0), Any(1), Any(2)]) failed. No function matches the given name and argument types 'nth_value(Int32, Int64, Int64)'. You might need to add explicit type casts. - Candidate functions: - nth_value() - nth_value(Any) - nth_value(Any, Any) # nth_value with wrong name