From 75c399ce7d4d5360140c64089dd7b05ffd7c49ef Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Wed, 10 Apr 2024 10:59:40 +0200 Subject: [PATCH 01/39] fix: reduce lock contention in `RepartitionExec::execute` (#10009) * fix: lock contention in `RepartitionExec::execute` The state is initialized ONCE for all partitions. However this may take a short while (on a very busy system 1ms or more). It is quite likely that multiple threads call `execute` at the same time, because we have just fanned out to the number "target partitions" which is likely set to the number of CPU cores which now all try to start to execute the plan at the same time. The solution is to not waste CPU circles in some futex lock but to tell the async runtime (= tokio) that we are performing work and the other threads should rather do something useful. This mostly just moves code around, no functional change intended. * docs: explain design choice Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- .../physical-plan/src/repartition/mod.rs | 338 +++++++++++------- 1 file changed, 206 insertions(+), 132 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c0dbf5164e19..2ed5da7ced20 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -36,6 +36,7 @@ use crate::repartition::distributor_channels::{ channels, partition_aware_channels, DistributionReceiver, DistributionSender, }; use crate::sorts::streaming_merge; +use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; use arrow::array::{ArrayRef, UInt64Builder}; @@ -48,7 +49,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; use futures::stream::Stream; -use futures::{FutureExt, StreamExt}; +use futures::{FutureExt, StreamExt, TryStreamExt}; use hashbrown::HashMap; use log::trace; use parking_lot::Mutex; @@ -77,6 +78,102 @@ struct RepartitionExecState { abort_helper: Arc>>, } +impl RepartitionExecState { + fn new( + input: Arc, + partitioning: Partitioning, + metrics: ExecutionPlanMetricsSet, + preserve_order: bool, + name: String, + context: Arc, + ) -> Self { + let num_input_partitions = input.output_partitioning().partition_count(); + let num_output_partitions = partitioning.partition_count(); + + let (txs, rxs) = if preserve_order { + let (txs, rxs) = + partition_aware_channels(num_input_partitions, num_output_partitions); + // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition + let txs = transpose(txs); + let rxs = transpose(rxs); + (txs, rxs) + } else { + // create one channel per *output* partition + // note we use a custom channel that ensures there is always data for each receiver + // but limits the amount of buffering if required. + let (txs, rxs) = channels(num_output_partitions); + // Clone sender for each input partitions + let txs = txs + .into_iter() + .map(|item| vec![item; num_input_partitions]) + .collect::>(); + let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); + (txs, rxs) + }; + + let mut channels = HashMap::with_capacity(txs.len()); + for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { + let reservation = Arc::new(Mutex::new( + MemoryConsumer::new(format!("{}[{partition}]", name)) + .register(context.memory_pool()), + )); + channels.insert(partition, (tx, rx, reservation)); + } + + // launch one async task per *input* partition + let mut spawned_tasks = Vec::with_capacity(num_input_partitions); + for i in 0..num_input_partitions { + let txs: HashMap<_, _> = channels + .iter() + .map(|(partition, (tx, _rx, reservation))| { + (*partition, (tx[i].clone(), Arc::clone(reservation))) + }) + .collect(); + + // TODO: metric input-output mapping is broken + let r_metrics = RepartitionMetrics::new(i, 0, &metrics); + + let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input( + input.clone(), + i, + txs.clone(), + partitioning.clone(), + r_metrics, + context.clone(), + )); + + // In a separate task, wait for each input to be done + // (and pass along any errors, including panic!s) + let wait_for_task = SpawnedTask::spawn(RepartitionExec::wait_for_task( + input_task, + txs.into_iter() + .map(|(partition, (tx, _reservation))| (partition, tx)) + .collect(), + )); + spawned_tasks.push(wait_for_task); + } + + Self { + channels, + abort_helper: Arc::new(spawned_tasks), + } + } +} + +/// Lazily initialized state +/// +/// Note that the state is initialized ONCE for all partitions by a single task(thread). +/// This may take a short while. It is also like that multiple threads +/// call execute at the same time, because we have just started "target partitions" tasks +/// which is commonly set to the number of CPU cores and all call execute at the same time. +/// +/// Thus, use a **tokio** `OnceCell` for this initialization so as not to waste CPU cycles +/// in a futex lock but instead allow other threads to do something useful. +/// +/// Uses a parking_lot `Mutex` to control other accesses as they are very short duration +/// (e.g. removing channels on completion) where the overhead of `await` is not warranted. +type LazyState = Arc>>; + /// A utility that can be used to partition batches based on [`Partitioning`] pub struct BatchPartitioner { state: BatchPartitionerState, @@ -298,7 +395,7 @@ pub struct RepartitionExec { /// Partitioning scheme to use partitioning: Partitioning, /// Inner state that is initialized when the first output stream is created. - state: Arc>, + state: LazyState, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Boolean flag to decide whether to preserve ordering. If true means @@ -453,134 +550,104 @@ impl ExecutionPlan for RepartitionExec { self.name(), partition ); - // lock mutexes - let mut state = self.state.lock(); - - let num_input_partitions = self.input.output_partitioning().partition_count(); - let num_output_partitions = self.partitioning.partition_count(); - - // if this is the first partition to be invoked then we need to set up initial state - if state.channels.is_empty() { - let (txs, rxs) = if self.preserve_order { - let (txs, rxs) = - partition_aware_channels(num_input_partitions, num_output_partitions); - // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition - let txs = transpose(txs); - let rxs = transpose(rxs); - (txs, rxs) - } else { - // create one channel per *output* partition - // note we use a custom channel that ensures there is always data for each receiver - // but limits the amount of buffering if required. - let (txs, rxs) = channels(num_output_partitions); - // Clone sender for each input partitions - let txs = txs - .into_iter() - .map(|item| vec![item; num_input_partitions]) - .collect::>(); - let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); - (txs, rxs) - }; - for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { - let reservation = Arc::new(Mutex::new( - MemoryConsumer::new(format!("{}[{partition}]", self.name())) - .register(context.memory_pool()), - )); - state.channels.insert(partition, (tx, rx, reservation)); - } - // launch one async task per *input* partition - let mut spawned_tasks = Vec::with_capacity(num_input_partitions); - for i in 0..num_input_partitions { - let txs: HashMap<_, _> = state - .channels - .iter() - .map(|(partition, (tx, _rx, reservation))| { - (*partition, (tx[i].clone(), Arc::clone(reservation))) - }) - .collect(); - - let r_metrics = RepartitionMetrics::new(i, partition, &self.metrics); - - let input_task = SpawnedTask::spawn(Self::pull_from_input( - self.input.clone(), - i, - txs.clone(), - self.partitioning.clone(), - r_metrics, - context.clone(), - )); - - // In a separate task, wait for each input to be done - // (and pass along any errors, including panic!s) - let wait_for_task = SpawnedTask::spawn(Self::wait_for_task( - input_task, - txs.into_iter() - .map(|(partition, (tx, _reservation))| (partition, tx)) - .collect(), - )); - spawned_tasks.push(wait_for_task); - } + let lazy_state = Arc::clone(&self.state); + let input = Arc::clone(&self.input); + let partitioning = self.partitioning.clone(); + let metrics = self.metrics.clone(); + let preserve_order = self.preserve_order; + let name = self.name().to_owned(); + let schema = self.schema(); + let schema_captured = Arc::clone(&schema); + + // Get existing ordering to use for merging + let sort_exprs = self.sort_exprs().unwrap_or(&[]).to_owned(); + + let stream = futures::stream::once(async move { + let num_input_partitions = input.output_partitioning().partition_count(); + + let input_captured = Arc::clone(&input); + let metrics_captured = metrics.clone(); + let name_captured = name.clone(); + let context_captured = Arc::clone(&context); + let state = lazy_state + .get_or_init(|| async move { + Mutex::new(RepartitionExecState::new( + input_captured, + partitioning, + metrics_captured, + preserve_order, + name_captured, + context_captured, + )) + }) + .await; - state.abort_helper = Arc::new(spawned_tasks) - } + // lock scope + let (mut rx, reservation, abort_helper) = { + // lock mutexes + let mut state = state.lock(); - trace!( - "Before returning stream in {}::execute for partition: {}", - self.name(), - partition - ); + // now return stream for the specified *output* partition which will + // read from the channel + let (_tx, rx, reservation) = state + .channels + .remove(&partition) + .expect("partition not used yet"); - // now return stream for the specified *output* partition which will - // read from the channel - let (_tx, mut rx, reservation) = state - .channels - .remove(&partition) - .expect("partition not used yet"); + (rx, reservation, Arc::clone(&state.abort_helper)) + }; - if self.preserve_order { - // Store streams from all the input partitions: - let input_streams = rx - .into_iter() - .map(|receiver| { - Box::pin(PerPartitionStream { - schema: self.schema(), - receiver, - drop_helper: Arc::clone(&state.abort_helper), - reservation: reservation.clone(), - }) as SendableRecordBatchStream - }) - .collect::>(); - // Note that receiver size (`rx.len()`) and `num_input_partitions` are same. - - // Get existing ordering to use for merging - let sort_exprs = self.sort_exprs().unwrap_or(&[]); - - // Merge streams (while preserving ordering) coming from - // input partitions to this partition: - let fetch = None; - let merge_reservation = - MemoryConsumer::new(format!("{}[Merge {partition}]", self.name())) - .register(context.memory_pool()); - streaming_merge( - input_streams, - self.schema(), - sort_exprs, - BaselineMetrics::new(&self.metrics, partition), - context.session_config().batch_size(), - fetch, - merge_reservation, - ) - } else { - Ok(Box::pin(RepartitionStream { - num_input_partitions, - num_input_partitions_processed: 0, - schema: self.input.schema(), - input: rx.swap_remove(0), - drop_helper: Arc::clone(&state.abort_helper), - reservation, - })) - } + trace!( + "Before returning stream in {}::execute for partition: {}", + name, + partition + ); + + if preserve_order { + // Store streams from all the input partitions: + let input_streams = rx + .into_iter() + .map(|receiver| { + Box::pin(PerPartitionStream { + schema: Arc::clone(&schema_captured), + receiver, + drop_helper: Arc::clone(&abort_helper), + reservation: reservation.clone(), + }) as SendableRecordBatchStream + }) + .collect::>(); + // Note that receiver size (`rx.len()`) and `num_input_partitions` are same. + + // Merge streams (while preserving ordering) coming from + // input partitions to this partition: + let fetch = None; + let merge_reservation = + MemoryConsumer::new(format!("{}[Merge {partition}]", name)) + .register(context.memory_pool()); + streaming_merge( + input_streams, + schema_captured, + &sort_exprs, + BaselineMetrics::new(&metrics, partition), + context.session_config().batch_size(), + fetch, + merge_reservation, + ) + } else { + Ok(Box::pin(RepartitionStream { + num_input_partitions, + num_input_partitions_processed: 0, + schema: input.schema(), + input: rx.swap_remove(0), + drop_helper: abort_helper, + reservation, + }) as SendableRecordBatchStream) + } + }) + .try_flatten(); + let stream = RecordBatchStreamAdapter::new(schema, stream); + Ok(Box::pin(stream)) } fn metrics(&self) -> Option { @@ -606,10 +673,7 @@ impl RepartitionExec { Ok(RepartitionExec { input, partitioning, - state: Arc::new(Mutex::new(RepartitionExecState { - channels: HashMap::new(), - abort_helper: Arc::new(Vec::new()), - })), + state: Default::default(), metrics: ExecutionPlanMetricsSet::new(), preserve_order, cache, @@ -951,6 +1015,7 @@ mod tests { use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use futures::FutureExt; + use tokio::task::JoinSet; #[tokio::test] async fn one_to_many_round_robin() -> Result<()> { @@ -1240,7 +1305,10 @@ mod tests { std::mem::drop(output_stream0); // Now, start sending input - input.wait().await; + let mut background_task = JoinSet::new(); + background_task.spawn(async move { + input.wait().await; + }); // output stream 1 should *not* error and have one of the input batches let batches = crate::common::collect(output_stream1).await.unwrap(); @@ -1277,7 +1345,10 @@ mod tests { let input = Arc::new(make_barrier_exec()); let exec = RepartitionExec::try_new(input.clone(), partitioning.clone()).unwrap(); let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); - input.wait().await; + let mut background_task = JoinSet::new(); + background_task.spawn(async move { + input.wait().await; + }); let batches_without_drop = crate::common::collect(output_stream1).await.unwrap(); // run some checks on the result @@ -1299,7 +1370,10 @@ mod tests { // now, purposely drop output stream 0 // *before* any outputs are produced std::mem::drop(output_stream0); - input.wait().await; + let mut background_task = JoinSet::new(); + background_task.spawn(async move { + input.wait().await; + }); let batches_with_drop = crate::common::collect(output_stream1).await.unwrap(); assert_eq!(batches_without_drop, batches_with_drop); From 843caea55af8991e9f31e2d17c3a3debbd3965ee Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 10 Apr 2024 12:14:31 +0200 Subject: [PATCH 02/39] chore(deps): update rstest requirement from 0.18.0 to 0.19.0 (#10021) Updates the requirements on [rstest](https://github.com/la10736/rstest) to permit the latest version. - [Release notes](https://github.com/la10736/rstest/releases) - [Changelog](https://github.com/la10736/rstest/blob/master/CHANGELOG.md) - [Commits](https://github.com/la10736/rstest/compare/v0.18.0...v0.18.2) --- updated-dependencies: - dependency-name: rstest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index e1e09d9893b1..c5a1aa9c8ef8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -101,7 +101,7 @@ object_store = { version = "0.9.1", default-features = false } parking_lot = "0.12" parquet = { version = "51.0.0", default-features = false, features = ["arrow", "async", "object_store"] } rand = "0.8" -rstest = "0.18.0" +rstest = "0.19.0" serde_json = "1" sqlparser = { version = "0.44.0", features = ["visitor"] } tempfile = "3" From 582050728914650c6d4340ca803a0e9af087d8ec Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 10 Apr 2024 06:26:44 -0400 Subject: [PATCH 03/39] Minor: Document LogicalPlan tree node transformations (#10010) * Document LogicalPlan tree node transformations * Add exists * touchups, add apply_subqueries, map_subqueries --- datafusion/core/src/lib.rs | 10 ++++-- datafusion/expr/src/logical_plan/mod.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 20 +++++++++++- datafusion/expr/src/logical_plan/tree_node.rs | 32 +++++++++++++++---- 4 files changed, 52 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index c213f4554fb8..b0e2b6fa9c09 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -296,11 +296,15 @@ //! A [`LogicalPlan`] is a Directed Acyclic Graph (DAG) of other //! [`LogicalPlan`]s, each potentially containing embedded [`Expr`]s. //! -//! [`Expr`]s can be rewritten using the [`TreeNode`] API and simplified using -//! [`ExprSimplifier`]. Examples of working with and executing `Expr`s can be found in the -//! [`expr_api`.rs] example +//! `LogicalPlan`s can be rewritten with [`TreeNode`] API, see the +//! [`tree_node module`] for more details. +//! +//! [`Expr`]s can also be rewritten with [`TreeNode`] API and simplified using +//! [`ExprSimplifier`]. Examples of working with and executing `Expr`s can be +//! found in the [`expr_api`.rs] example //! //! [`TreeNode`]: datafusion_common::tree_node::TreeNode +//! [`tree_node module`]: datafusion_expr::logical_plan::tree_node //! [`ExprSimplifier`]: crate::optimizer::simplify_expressions::ExprSimplifier //! [`expr_api`.rs]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/expr_api.rs //! diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index a1fe7a6f0a51..034440643e51 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -22,7 +22,7 @@ pub mod dml; mod extension; mod plan; mod statement; -mod tree_node; +pub mod tree_node; pub use builder::{ build_join_schema, table_scan, union, wrap_projection_for_join_if_necessary, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 02d65973a50b..7bad034a11ea 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -68,6 +68,11 @@ pub use datafusion_common::{JoinConstraint, JoinType}; /// an output relation (table) with a (potentially) different /// schema. A plan represents a dataflow tree where data flows /// from leaves up to the root to produce the query result. +/// +/// # See also: +/// * [`tree_node`]: visiting and rewriting API +/// +/// [`tree_node`]: crate::logical_plan::tree_node #[derive(Clone, PartialEq, Eq, Hash)] pub enum LogicalPlan { /// Evaluates an arbitrary list of expressions (essentially a @@ -238,7 +243,10 @@ impl LogicalPlan { } /// Returns all expressions (non-recursively) evaluated by the current - /// logical plan node. This does not include expressions in any children + /// logical plan node. This does not include expressions in any children. + /// + /// Note this method `clone`s all the expressions. When possible, the + /// [`tree_node`] API should be used instead of this API. /// /// The returned expressions do not necessarily represent or even /// contributed to the output schema of this node. For example, @@ -248,6 +256,8 @@ impl LogicalPlan { /// The expressions do contain all the columns that are used by this plan, /// so if there are columns not referenced by these expressions then /// DataFusion's optimizer attempts to optimize them away. + /// + /// [`tree_node`]: crate::logical_plan::tree_node pub fn expressions(self: &LogicalPlan) -> Vec { let mut exprs = vec![]; self.apply_expressions(|e| { @@ -773,10 +783,16 @@ impl LogicalPlan { /// Returns a new `LogicalPlan` based on `self` with inputs and /// expressions replaced. /// + /// Note this method creates an entirely new node, which requires a large + /// amount of clone'ing. When possible, the [`tree_node`] API should be used + /// instead of this API. + /// /// The exprs correspond to the same order of expressions returned /// by [`Self::expressions`]. This function is used by optimizers /// to rewrite plans using the following pattern: /// + /// [`tree_node`]: crate::logical_plan::tree_node + /// /// ```text /// let new_inputs = optimize_children(..., plan, props); /// @@ -1367,6 +1383,7 @@ macro_rules! handle_transform_recursion_up { } impl LogicalPlan { + /// Visits a plan similarly to [`Self::visit`], but including embedded subqueries. pub fn visit_with_subqueries>( &self, visitor: &mut V, @@ -1380,6 +1397,7 @@ impl LogicalPlan { .visit_parent(|| visitor.f_up(self)) } + /// Rewrites a plan similarly t [`Self::visit`], but including embedded subqueries. pub fn rewrite_with_subqueries>( self, rewriter: &mut R, diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index ce26cac7970b..415343f88685 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -15,17 +15,35 @@ // specific language governing permissions and limitations // under the License. -//! Tree node implementation for logical plan - +//! [`TreeNode`] based visiting and rewriting for [`LogicalPlan`]s +//! +//! Visiting (read only) APIs +//! * [`LogicalPlan::visit`]: recursively visit the node and all of its inputs +//! * [`LogicalPlan::visit_with_subqueries`]: recursively visit the node and all of its inputs, including subqueries +//! * [`LogicalPlan::apply_children`]: recursively visit all inputs of this node +//! * [`LogicalPlan::apply_expressions`]: (non recursively) visit all expressions of this node +//! * [`LogicalPlan::apply_subqueries`]: (non recursively) visit all subqueries of this node +//! * [`LogicalPlan::apply_with_subqueries`]: recursively visit all inputs and embedded subqueries. +//! +//! Rewriting (update) APIs: +//! * [`LogicalPlan::exists`]: search for an expression in a plan +//! * [`LogicalPlan::rewrite`]: recursively rewrite the node and all of its inputs +//! * [`LogicalPlan::map_children`]: recursively rewrite all inputs of this node +//! * [`LogicalPlan::map_expressions`]: (non recursively) visit all expressions of this node +//! * [`LogicalPlan::map_subqueries`]: (non recursively) rewrite all subqueries of this node +//! * [`LogicalPlan::rewrite_with_subqueries`]: recursively rewrite the node and all of its inputs, including subqueries +//! +//! (Re)creation APIs (these require substantial cloning and thus are slow): +//! * [`LogicalPlan::with_new_exprs`]: Create a new plan with different expressions +//! * [`LogicalPlan::expressions`]: Return a copy of the plan's expressions use crate::{ - Aggregate, Analyze, CreateMemoryTable, CreateView, CrossJoin, DdlStatement, Distinct, - DistinctOn, DmlStatement, Explain, Extension, Filter, Join, Limit, LogicalPlan, - Prepare, Projection, RecursiveQuery, Repartition, Sort, Subquery, SubqueryAlias, - Union, Unnest, Window, + dml::CopyTo, Aggregate, Analyze, CreateMemoryTable, CreateView, CrossJoin, + DdlStatement, Distinct, DistinctOn, DmlStatement, Explain, Extension, Filter, Join, + Limit, LogicalPlan, Prepare, Projection, RecursiveQuery, Repartition, Sort, Subquery, + SubqueryAlias, Union, Unnest, Window, }; use std::sync::Arc; -use crate::dml::CopyTo; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, }; From 03d8ba1f0d94bac6bb8bb33e95f00f9f6fb5275a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 10 Apr 2024 09:27:35 -0400 Subject: [PATCH 04/39] Refactor `Optimizer` to use owned plans and `TreeNode` API (10% faster planning) (#9948) * Rewrite Optimizer to use TreeNode API * fmt --- datafusion-examples/examples/rewrite_expr.rs | 2 +- datafusion/common/src/tree_node.rs | 12 +- datafusion/core/src/execution/context/mod.rs | 4 +- .../core/tests/optimizer_integration.rs | 2 +- .../src/decorrelate_predicate_subquery.rs | 96 ++--- .../src/eliminate_duplicated_expr.rs | 6 +- datafusion/optimizer/src/eliminate_filter.rs | 14 +- datafusion/optimizer/src/eliminate_join.rs | 6 +- datafusion/optimizer/src/eliminate_limit.rs | 32 +- .../optimizer/src/eliminate_nested_union.rs | 22 +- .../optimizer/src/eliminate_one_union.rs | 6 +- .../optimizer/src/eliminate_outer_join.rs | 12 +- .../src/extract_equijoin_predicate.rs | 18 +- .../optimizer/src/filter_null_join_keys.rs | 14 +- .../optimizer/src/optimize_projections.rs | 50 +-- datafusion/optimizer/src/optimizer.rs | 348 +++++++++--------- .../optimizer/src/propagate_empty_relation.rs | 22 +- datafusion/optimizer/src/push_down_filter.rs | 151 ++++---- datafusion/optimizer/src/push_down_limit.rs | 70 ++-- .../optimizer/src/push_down_projection.rs | 76 ++-- .../src/replace_distinct_aggregate.rs | 4 +- .../optimizer/src/scalar_subquery_to_join.rs | 28 +- .../src/single_distinct_to_groupby.rs | 40 +- datafusion/optimizer/src/test/mod.rs | 68 ++-- .../optimizer/tests/optimizer_integration.rs | 6 +- datafusion/sqllogictest/test_files/join.slt | 2 +- 26 files changed, 535 insertions(+), 576 deletions(-) diff --git a/datafusion-examples/examples/rewrite_expr.rs b/datafusion-examples/examples/rewrite_expr.rs index 541448ebf149..dcebbb55fb66 100644 --- a/datafusion-examples/examples/rewrite_expr.rs +++ b/datafusion-examples/examples/rewrite_expr.rs @@ -59,7 +59,7 @@ pub fn main() -> Result<()> { // then run the optimizer with our custom rule let optimizer = Optimizer::with_rules(vec![Arc::new(MyOptimizerRule {})]); - let optimized_plan = optimizer.optimize(&analyzed_plan, &config, observe)?; + let optimized_plan = optimizer.optimize(analyzed_plan, &config, observe)?; println!( "Optimized Logical Plan:\n\n{}\n", optimized_plan.display_indent() diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index bb268e048d9a..dff22d495958 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -56,6 +56,9 @@ pub trait TreeNode: Sized { /// Visit the tree node using the given [`TreeNodeVisitor`], performing a /// depth-first walk of the node and its children. /// + /// See also: + /// * [`Self::rewrite`] to rewrite owned `TreeNode`s + /// /// Consider the following tree structure: /// ```text /// ParentNode @@ -93,6 +96,9 @@ pub trait TreeNode: Sized { /// Implements the [visitor pattern](https://en.wikipedia.org/wiki/Visitor_pattern) for /// recursively transforming [`TreeNode`]s. /// + /// See also: + /// * [`Self::visit`] for inspecting (without modification) `TreeNode`s + /// /// Consider the following tree structure: /// ```text /// ParentNode @@ -310,13 +316,15 @@ pub trait TreeNode: Sized { } /// Apply the closure `F` to the node's children. + /// + /// See `mutate_children` for rewriting in place fn apply_children Result>( &self, f: F, ) -> Result; - /// Apply transform `F` to the node's children. Note that the transform `F` - /// might have a direction (pre-order or post-order). + /// Apply transform `F` to potentially rewrite the node's children. Note + /// that the transform `F` might have a direction (pre-order or post-order). fn map_children Result>>( self, f: F, diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 9e48c7b8a6f2..5cf8969aa46d 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1881,7 +1881,7 @@ impl SessionState { // optimize the child plan, capturing the output of each optimizer let optimized_plan = self.optimizer.optimize( - &analyzed_plan, + analyzed_plan, self, |optimized_plan, optimizer| { let optimizer_name = optimizer.name().to_string(); @@ -1911,7 +1911,7 @@ impl SessionState { let analyzed_plan = self.analyzer .execute_and_check(plan, self.options(), |_, _| {})?; - self.optimizer.optimize(&analyzed_plan, self, |_, _| {}) + self.optimizer.optimize(analyzed_plan, self, |_, _| {}) } } diff --git a/datafusion/core/tests/optimizer_integration.rs b/datafusion/core/tests/optimizer_integration.rs index 60010bdddfb8..6e938361ddb4 100644 --- a/datafusion/core/tests/optimizer_integration.rs +++ b/datafusion/core/tests/optimizer_integration.rs @@ -110,7 +110,7 @@ fn test_sql(sql: &str) -> Result { let optimizer = Optimizer::new(); // analyze and optimize the logical plan let plan = analyzer.execute_and_check(&plan, config.options(), |_, _| {})?; - optimizer.optimize(&plan, &config, |_, _| {}) + optimizer.optimize(plan, &config, |_, _| {}) } #[derive(Default)] diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 019e7507b122..d9fc5a6ce261 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -338,7 +338,7 @@ mod tests { Operator, }; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), plan, @@ -378,7 +378,7 @@ mod tests { \n SubqueryAlias: __correlated_sq_2 [c:UInt32]\ \n Projection: sq_2.c [c:UInt32]\ \n TableScan: sq_2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for IN subquery with additional AND filter @@ -404,7 +404,7 @@ mod tests { \n Projection: sq.c [c:UInt32]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for IN subquery with additional OR filter @@ -430,7 +430,7 @@ mod tests { \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -458,7 +458,7 @@ mod tests { \n Projection: sq2.c [c:UInt32]\ \n TableScan: sq2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for nested IN subqueries @@ -487,7 +487,7 @@ mod tests { \n Projection: sq_nested.c [c:UInt32]\ \n TableScan: sq_nested [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for filter input modification in case filter not supported @@ -519,7 +519,7 @@ mod tests { \n Projection: sq_inner.c [c:UInt32]\ \n TableScan: sq_inner [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test multiple correlated subqueries @@ -557,7 +557,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -607,7 +607,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -642,7 +642,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -675,7 +675,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -706,7 +706,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -739,7 +739,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -772,7 +772,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -806,7 +806,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); @@ -863,7 +863,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -896,7 +896,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -962,7 +962,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1000,7 +1000,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1030,7 +1030,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1054,7 +1054,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1078,7 +1078,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1107,7 +1107,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1142,7 +1142,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1178,7 +1178,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1224,7 +1224,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1255,7 +1255,7 @@ mod tests { assert_optimized_plan_eq_display_indent( Arc::new(DecorrelatePredicateSubquery::new()), - &plan, + plan, expected, ); Ok(()) @@ -1289,7 +1289,7 @@ mod tests { \n SubqueryAlias: __correlated_sq_2 [o_custkey:Int64]\ \n Projection: orders.o_custkey [o_custkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test recursive correlated subqueries @@ -1332,7 +1332,7 @@ mod tests { \n SubqueryAlias: __correlated_sq_2 [l_orderkey:Int64]\ \n Projection: lineitem.l_orderkey [l_orderkey:Int64]\ \n TableScan: lineitem [l_orderkey:Int64, l_partkey:Int64, l_suppkey:Int64, l_linenumber:Int32, l_quantity:Float64, l_extendedprice:Float64]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists subquery filter with additional subquery filters @@ -1362,7 +1362,7 @@ mod tests { \n Filter: orders.o_orderkey = Int32(1) [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1387,7 +1387,7 @@ mod tests { \n Projection: orders.o_custkey [o_custkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for exists subquery with both columns in schema @@ -1405,7 +1405,7 @@ mod tests { .project(vec![col("customer.c_custkey")])? .build()?; - assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), &plan) + assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), plan) } /// Test for correlated exists subquery not equal @@ -1433,7 +1433,7 @@ mod tests { \n Projection: orders.o_custkey [o_custkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists subquery less than @@ -1461,7 +1461,7 @@ mod tests { \n Projection: orders.o_custkey [o_custkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists subquery filter with subquery disjunction @@ -1490,7 +1490,7 @@ mod tests { \n Projection: orders.o_custkey, orders.o_orderkey [o_custkey:Int64, o_orderkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists without projection @@ -1516,7 +1516,7 @@ mod tests { \n SubqueryAlias: __correlated_sq_1 [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists expressions @@ -1544,7 +1544,7 @@ mod tests { \n Projection: orders.o_custkey + Int32(1), orders.o_custkey [orders.o_custkey + Int32(1):Int64, o_custkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists subquery filter with additional filters @@ -1572,7 +1572,7 @@ mod tests { \n Projection: orders.o_custkey [o_custkey:Int64]\ \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists subquery filter with disjustions @@ -1599,7 +1599,7 @@ mod tests { TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N] TableScan: customer [c_custkey:Int64, c_name:Utf8]"#; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for correlated EXISTS subquery filter @@ -1624,7 +1624,7 @@ mod tests { \n Projection: sq.c, sq.a [c:UInt32, a:UInt32]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } /// Test for single exists subquery filter @@ -1636,7 +1636,7 @@ mod tests { .project(vec![col("test.b")])? .build()?; - assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), &plan) + assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), plan) } /// Test for single NOT exists subquery filter @@ -1648,7 +1648,7 @@ mod tests { .project(vec![col("test.b")])? .build()?; - assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), &plan) + assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), plan) } #[test] @@ -1687,7 +1687,7 @@ mod tests { \n Projection: sq2.c, sq2.a [c:UInt32, a:UInt32]\ \n TableScan: sq2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1713,7 +1713,7 @@ mod tests { \n Projection: UInt32(1), sq.a [UInt32(1):UInt32, a:UInt32]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1739,7 +1739,7 @@ mod tests { \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1767,7 +1767,7 @@ mod tests { \n Projection: sq.c, sq.a [c:UInt32, a:UInt32]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1795,7 +1795,7 @@ mod tests { \n Projection: sq.b + sq.c, sq.a [sq.b + sq.c:UInt32, a:UInt32]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1823,6 +1823,6 @@ mod tests { \n Projection: UInt32(1), sq.c, sq.a [UInt32(1):UInt32, c:UInt32, a:UInt32]\ \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_duplicated_expr.rs b/datafusion/optimizer/src/eliminate_duplicated_expr.rs index 349d4d8878e0..ee44a328f8b3 100644 --- a/datafusion/optimizer/src/eliminate_duplicated_expr.rs +++ b/datafusion/optimizer/src/eliminate_duplicated_expr.rs @@ -116,7 +116,7 @@ mod tests { use datafusion_expr::{col, logical_plan::builder::LogicalPlanBuilder}; use std::sync::Arc; - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { crate::test::assert_optimized_plan_eq( Arc::new(EliminateDuplicatedExpr::new()), plan, @@ -134,7 +134,7 @@ mod tests { let expected = "Limit: skip=5, fetch=10\ \n Sort: test.a, test.b, test.c\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -153,6 +153,6 @@ mod tests { let expected = "Limit: skip=5, fetch=10\ \n Sort: test.a ASC NULLS FIRST, test.b ASC NULLS LAST\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_filter.rs b/datafusion/optimizer/src/eliminate_filter.rs index 9411dc192beb..2bf5cfa30390 100644 --- a/datafusion/optimizer/src/eliminate_filter.rs +++ b/datafusion/optimizer/src/eliminate_filter.rs @@ -91,7 +91,7 @@ mod tests { use crate::test::*; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(EliminateFilter::new()), plan, expected) } @@ -107,7 +107,7 @@ mod tests { // No aggregate / scan / limit let expected = "EmptyRelation"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -122,7 +122,7 @@ mod tests { // No aggregate / scan / limit let expected = "EmptyRelation"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -144,7 +144,7 @@ mod tests { \n EmptyRelation\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -159,7 +159,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -182,7 +182,7 @@ mod tests { \n TableScan: test\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -205,6 +205,6 @@ mod tests { // Filter is removed let expected = "Projection: test.a\ \n EmptyRelation"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_join.rs b/datafusion/optimizer/src/eliminate_join.rs index e685229c61b2..caf45dda9896 100644 --- a/datafusion/optimizer/src/eliminate_join.rs +++ b/datafusion/optimizer/src/eliminate_join.rs @@ -83,7 +83,7 @@ mod tests { use datafusion_expr::{logical_plan::builder::LogicalPlanBuilder, Expr, LogicalPlan}; use std::sync::Arc; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(EliminateJoin::new()), plan, expected) } @@ -98,7 +98,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -115,6 +115,6 @@ mod tests { CrossJoin:\ \n EmptyRelation\ \n EmptyRelation"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index fb5d0d17b839..39231d784e00 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -94,24 +94,19 @@ mod tests { use crate::push_down_limit::PushDownLimit; - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + fn assert_optimized_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { let optimizer = Optimizer::with_rules(vec![Arc::new(EliminateLimit::new())]); - let optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - )? - .unwrap_or_else(|| plan.clone()); + let optimized_plan = + optimizer.optimize(plan, &OptimizerContext::new(), observe)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); - assert_eq!(plan.schema(), optimized_plan.schema()); Ok(()) } fn assert_optimized_plan_eq_with_pushdown( - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) -> Result<()> { fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} @@ -125,7 +120,6 @@ mod tests { .expect("failed to optimize plan"); let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); - assert_eq!(plan.schema(), optimized_plan.schema()); Ok(()) } @@ -138,7 +132,7 @@ mod tests { .build()?; // No aggregate / scan / limit let expected = "EmptyRelation"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -158,7 +152,7 @@ mod tests { \n EmptyRelation\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -172,7 +166,7 @@ mod tests { // No aggregate / scan / limit let expected = "EmptyRelation"; - assert_optimized_plan_eq_with_pushdown(&plan, expected) + assert_optimized_plan_eq_with_pushdown(plan, expected) } #[test] @@ -192,7 +186,7 @@ mod tests { \n Limit: skip=0, fetch=2\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_eq_with_pushdown(&plan, expected) + assert_optimized_plan_eq_with_pushdown(plan, expected) } #[test] @@ -210,7 +204,7 @@ mod tests { \n Limit: skip=0, fetch=2\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -228,7 +222,7 @@ mod tests { \n Limit: skip=2, fetch=1\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -250,7 +244,7 @@ mod tests { \n Limit: skip=2, fetch=1\ \n TableScan: test\ \n TableScan: test1"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -263,6 +257,6 @@ mod tests { let expected = "Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index 924a0853418c..da2a6a17214e 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -114,7 +114,7 @@ mod tests { ]) } - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(EliminateNestedUnion::new()), plan, expected) } @@ -131,7 +131,7 @@ mod tests { Union\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -147,7 +147,7 @@ mod tests { \n Union\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -167,7 +167,7 @@ mod tests { \n TableScan: table\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -188,7 +188,7 @@ mod tests { \n TableScan: table\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -210,7 +210,7 @@ mod tests { \n TableScan: table\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -230,7 +230,7 @@ mod tests { \n TableScan: table\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // We don't need to use project_with_column_index in logical optimizer, @@ -261,7 +261,7 @@ mod tests { \n TableScan: table\ \n Projection: table.id AS id, table.key, table.value\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -291,7 +291,7 @@ mod tests { \n TableScan: table\ \n Projection: table.id AS id, table.key, table.value\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -337,7 +337,7 @@ mod tests { \n TableScan: table_1\ \n Projection: CAST(table_1.id AS Int64) AS id, table_1.key, CAST(table_1.value AS Float64) AS value\ \n TableScan: table_1"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -384,6 +384,6 @@ mod tests { \n TableScan: table_1\ \n Projection: CAST(table_1.id AS Int64) AS id, table_1.key, CAST(table_1.value AS Float64) AS value\ \n TableScan: table_1"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 63c3e789daa6..95a3370ab1b5 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -76,7 +76,7 @@ mod tests { ]) } - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_with_rules( vec![Arc::new(EliminateOneUnion::new())], plan, @@ -97,7 +97,7 @@ mod tests { Union\ \n TableScan: table\ \n TableScan: table"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -113,6 +113,6 @@ mod tests { }); let expected = "TableScan: table"; - assert_optimized_plan_equal(&single_union_plan, expected) + assert_optimized_plan_equal(single_union_plan, expected) } } diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index a004da2bff19..63b8b887bb32 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -306,7 +306,7 @@ mod tests { Operator::{And, Or}, }; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(EliminateOuterJoin::new()), plan, expected) } @@ -330,7 +330,7 @@ mod tests { \n Left Join: t1.a = t2.a\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -353,7 +353,7 @@ mod tests { \n Inner Join: t1.a = t2.a\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -380,7 +380,7 @@ mod tests { \n Inner Join: t1.a = t2.a\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -407,7 +407,7 @@ mod tests { \n Inner Join: t1.a = t2.a\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -434,6 +434,6 @@ mod tests { \n Inner Join: t1.a = t2.a\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 4cfcd07b47d9..60b9ba3031a1 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -164,7 +164,7 @@ mod tests { col, lit, logical_plan::builder::LogicalPlanBuilder, JoinType, }; - fn assert_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_display_indent( Arc::new(ExtractEquijoinPredicate {}), plan, @@ -186,7 +186,7 @@ mod tests { \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -205,7 +205,7 @@ mod tests { \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -228,7 +228,7 @@ mod tests { \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -255,7 +255,7 @@ mod tests { \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -281,7 +281,7 @@ mod tests { \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -318,7 +318,7 @@ mod tests { \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t3 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -351,7 +351,7 @@ mod tests { \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t3 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } #[test] @@ -375,6 +375,6 @@ mod tests { \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ \n TableScan: t2 [a:UInt32, b:UInt32, c:UInt32]"; - assert_plan_eq(&plan, expected) + assert_plan_eq(plan, expected) } } diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index 16039b182bb2..fcf85327fdb0 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -116,7 +116,7 @@ mod tests { use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{col, lit, logical_plan::JoinType, LogicalPlanBuilder}; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(FilterNullJoinKeys {}), plan, expected) } @@ -128,7 +128,7 @@ mod tests { \n Filter: t1.optional_id IS NOT NULL\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -139,7 +139,7 @@ mod tests { \n Filter: t1.optional_id IS NOT NULL\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -176,7 +176,7 @@ mod tests { \n Filter: t1.optional_id IS NOT NULL\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -197,7 +197,7 @@ mod tests { \n Filter: t1.optional_id + UInt32(1) IS NOT NULL\ \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -218,7 +218,7 @@ mod tests { \n TableScan: t1\ \n Filter: t2.optional_id + UInt32(1) IS NOT NULL\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -241,7 +241,7 @@ mod tests { \n TableScan: t1\ \n Filter: t2.optional_id + UInt32(1) IS NOT NULL\ \n TableScan: t2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } fn build_plan( diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 69905c990a7f..6967b28f3037 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -941,7 +941,7 @@ mod tests { UserDefinedLogicalNodeCore, }; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(OptimizeProjections::new()), plan, expected) } @@ -1090,7 +1090,7 @@ mod tests { let expected = "Projection: Int32(1) + test.a\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1104,7 +1104,7 @@ mod tests { let expected = "Projection: Int32(1) + test.a\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1117,7 +1117,7 @@ mod tests { let expected = "Projection: test.a AS alias\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1130,7 +1130,7 @@ mod tests { let expected = "Projection: test.a AS alias\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1152,7 +1152,7 @@ mod tests { \n Projection: \ \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int32(1))]]\ \n TableScan: ?table? projection=[]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1175,7 +1175,7 @@ mod tests { .build()?; let expected = "Projection: (?table?.s)[x]\ \n TableScan: ?table? projection=[s]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1187,7 +1187,7 @@ mod tests { let expected = "Projection: (- test.a)\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1199,7 +1199,7 @@ mod tests { let expected = "Projection: test.a IS NULL\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1211,7 +1211,7 @@ mod tests { let expected = "Projection: test.a IS NOT NULL\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1223,7 +1223,7 @@ mod tests { let expected = "Projection: test.a IS TRUE\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1235,7 +1235,7 @@ mod tests { let expected = "Projection: test.a IS NOT TRUE\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1247,7 +1247,7 @@ mod tests { let expected = "Projection: test.a IS FALSE\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1259,7 +1259,7 @@ mod tests { let expected = "Projection: test.a IS NOT FALSE\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1271,7 +1271,7 @@ mod tests { let expected = "Projection: test.a IS UNKNOWN\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1283,7 +1283,7 @@ mod tests { let expected = "Projection: test.a IS NOT UNKNOWN\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1295,7 +1295,7 @@ mod tests { let expected = "Projection: NOT test.a\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1307,7 +1307,7 @@ mod tests { let expected = "Projection: TRY_CAST(test.a AS Float64)\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1323,7 +1323,7 @@ mod tests { let expected = "Projection: test.a SIMILAR TO Utf8(\"[0-9]\")\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1335,7 +1335,7 @@ mod tests { let expected = "Projection: test.a BETWEEN Int32(1) AND Int32(3)\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Test outer projection isn't discarded despite the same schema as inner @@ -1356,7 +1356,7 @@ mod tests { let expected = "Projection: test.a, CASE WHEN test.a = Int32(1) THEN Int32(10) ELSE d END AS d\ \n Projection: test.a, Int32(0) AS d\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Since only column `a` is referred at the output. Scan should only contain projection=[a]. @@ -1377,7 +1377,7 @@ mod tests { let expected = "Projection: test.a, Int32(0) AS d\ \n NoOpUserDefined\ \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Only column `a` is referred at the output. However, User defined node itself uses column `b` @@ -1404,7 +1404,7 @@ mod tests { let expected = "Projection: test.a, Int32(0) AS d\ \n NoOpUserDefined\ \n TableScan: test projection=[a, b]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Only column `a` is referred at the output. However, User defined node itself uses expression `b+c` @@ -1439,7 +1439,7 @@ mod tests { let expected = "Projection: test.a, Int32(0) AS d\ \n NoOpUserDefined\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Columns `l.a`, `l.c`, `r.a` is referred at the output. @@ -1464,6 +1464,6 @@ mod tests { \n UserDefinedCrossJoin\ \n TableScan: l projection=[a, c]\ \n TableScan: r projection=[a]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 03ff402c3e3f..032f9c57321c 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -20,6 +20,16 @@ use std::collections::HashSet; use std::sync::Arc; +use chrono::{DateTime, Utc}; +use log::{debug, warn}; + +use datafusion_common::alias::AliasGenerator; +use datafusion_common::config::ConfigOptions; +use datafusion_common::instant::Instant; +use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; +use datafusion_common::{DFSchema, DataFusionError, Result}; +use datafusion_expr::logical_plan::LogicalPlan; + use crate::common_subexpr_eliminate::CommonSubexprEliminate; use crate::decorrelate_predicate_subquery::DecorrelatePredicateSubquery; use crate::eliminate_cross_join::EliminateCrossJoin; @@ -45,15 +55,6 @@ use crate::single_distinct_to_groupby::SingleDistinctToGroupBy; use crate::unwrap_cast_in_comparison::UnwrapCastInComparison; use crate::utils::log_plan; -use datafusion_common::alias::AliasGenerator; -use datafusion_common::config::ConfigOptions; -use datafusion_common::instant::Instant; -use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::logical_plan::LogicalPlan; - -use chrono::{DateTime, Utc}; -use log::{debug, warn}; - /// `OptimizerRule`s transforms one [`LogicalPlan`] into another which /// computes the same results, but in a potentially more efficient /// way. If there are no suitable transformations for the input plan, @@ -184,41 +185,15 @@ pub struct Optimizer { pub rules: Vec>, } -/// If a rule is with `ApplyOrder`, it means the optimizer will derive to handle children instead of -/// recursively handling in rule. -/// We just need handle a subtree pattern itself. -/// -/// Notice: **sometime** result after optimize still can be optimized, we need apply again. +/// Specifies how recursion for an `OptimizerRule` should be handled. /// -/// Usage Example: Merge Limit (subtree pattern is: Limit-Limit) -/// ```rust -/// use datafusion_expr::{Limit, LogicalPlan, LogicalPlanBuilder}; -/// use datafusion_common::Result; -/// fn merge_limit(parent: &Limit, child: &Limit) -> LogicalPlan { -/// // just for run -/// return parent.input.as_ref().clone(); -/// } -/// fn try_optimize(plan: &LogicalPlan) -> Result> { -/// match plan { -/// LogicalPlan::Limit(limit) => match limit.input.as_ref() { -/// LogicalPlan::Limit(child_limit) => { -/// // merge limit ... -/// let optimized_plan = merge_limit(limit, child_limit); -/// // due to optimized_plan may be optimized again, -/// // for example: plan is Limit-Limit-Limit -/// Ok(Some( -/// try_optimize(&optimized_plan)? -/// .unwrap_or_else(|| optimized_plan.clone()), -/// )) -/// } -/// _ => Ok(None), -/// }, -/// _ => Ok(None), -/// } -/// } -/// ``` +/// * `Some(apply_order)`: The Optimizer will recursively apply the rule to the plan. +/// * `None`: the rule must handle any required recursion itself. +#[derive(Debug, Clone, Copy, PartialEq)] pub enum ApplyOrder { + /// Apply the rule to the node before its inputs TopDown, + /// Apply the rule to the node after its inputs BottomUp, } @@ -274,22 +249,78 @@ impl Optimizer { pub fn with_rules(rules: Vec>) -> Self { Self { rules } } +} + +/// Recursively rewrites LogicalPlans +struct Rewriter<'a> { + apply_order: ApplyOrder, + rule: &'a dyn OptimizerRule, + config: &'a dyn OptimizerConfig, +} +impl<'a> Rewriter<'a> { + fn new( + apply_order: ApplyOrder, + rule: &'a dyn OptimizerRule, + config: &'a dyn OptimizerConfig, + ) -> Self { + Self { + apply_order, + rule, + config, + } + } +} + +impl<'a> TreeNodeRewriter for Rewriter<'a> { + type Node = LogicalPlan; + + fn f_down(&mut self, node: LogicalPlan) -> Result> { + if self.apply_order == ApplyOrder::TopDown { + optimize_plan_node(node, self.rule, self.config) + } else { + Ok(Transformed::no(node)) + } + } + + fn f_up(&mut self, node: LogicalPlan) -> Result> { + if self.apply_order == ApplyOrder::BottomUp { + optimize_plan_node(node, self.rule, self.config) + } else { + Ok(Transformed::no(node)) + } + } +} + +/// Invokes the Optimizer rule to rewrite the LogicalPlan in place. +fn optimize_plan_node( + plan: LogicalPlan, + rule: &dyn OptimizerRule, + config: &dyn OptimizerConfig, +) -> Result> { + // TODO: add API to OptimizerRule to allow rewriting by ownership + rule.try_optimize(&plan, config) + .map(|maybe_plan| match maybe_plan { + Some(new_plan) => Transformed::yes(new_plan), + None => Transformed::no(plan), + }) +} + +impl Optimizer { /// Optimizes the logical plan by applying optimizer rules, and /// invoking observer function after each call pub fn optimize( &self, - plan: &LogicalPlan, + plan: LogicalPlan, config: &dyn OptimizerConfig, mut observer: F, ) -> Result where F: FnMut(&LogicalPlan, &dyn OptimizerRule), { - let options = config.options(); - let mut new_plan = plan.clone(); - let start_time = Instant::now(); + let options = config.options(); + let mut new_plan = plan; let mut previous_plans = HashSet::with_capacity(16); previous_plans.insert(LogicalPlanSignature::new(&new_plan)); @@ -299,44 +330,71 @@ impl Optimizer { log_plan(&format!("Optimizer input (pass {i})"), &new_plan); for rule in &self.rules { - let result = - self.optimize_recursively(rule, &new_plan, config) - .and_then(|plan| { - if let Some(plan) = &plan { - assert_schema_is_the_same(rule.name(), plan, &new_plan)?; - } - Ok(plan) - }); - match result { - Ok(Some(plan)) => { - new_plan = plan; - observer(&new_plan, rule.as_ref()); - log_plan(rule.name(), &new_plan); - } - Ok(None) => { + // If skipping failed rules, copy plan before attempting to rewrite + // as rewriting is destructive + let prev_plan = options + .optimizer + .skip_failed_rules + .then(|| new_plan.clone()); + + let starting_schema = new_plan.schema().clone(); + + let result = match rule.apply_order() { + // optimizer handles recursion + Some(apply_order) => new_plan.rewrite(&mut Rewriter::new( + apply_order, + rule.as_ref(), + config, + )), + // rule handles recursion itself + None => optimize_plan_node(new_plan, rule.as_ref(), config), + } + // verify the rule didn't change the schema + .and_then(|tnr| { + assert_schema_is_the_same(rule.name(), &starting_schema, &tnr.data)?; + Ok(tnr) + }); + + // Handle results + match (result, prev_plan) { + // OptimizerRule was successful + ( + Ok(Transformed { + data, transformed, .. + }), + _, + ) => { + new_plan = data; observer(&new_plan, rule.as_ref()); - debug!( - "Plan unchanged by optimizer rule '{}' (pass {})", - rule.name(), - i - ); + if transformed { + log_plan(rule.name(), &new_plan); + } else { + debug!( + "Plan unchanged by optimizer rule '{}' (pass {})", + rule.name(), + i + ); + } } - Err(e) => { - if options.optimizer.skip_failed_rules { - // Note to future readers: if you see this warning it signals a - // bug in the DataFusion optimizer. Please consider filing a ticket - // https://github.com/apache/arrow-datafusion - warn!( + // OptimizerRule was unsuccessful, but skipped failed rules is on + // so use the previous plan + (Err(e), Some(orig_plan)) => { + // Note to future readers: if you see this warning it signals a + // bug in the DataFusion optimizer. Please consider filing a ticket + // https://github.com/apache/arrow-datafusion + warn!( "Skipping optimizer rule '{}' due to unexpected error: {}", rule.name(), e ); - } else { - return Err(DataFusionError::Context( - format!("Optimizer rule '{}' failed", rule.name(),), - Box::new(e), - )); - } + new_plan = orig_plan; + } + // OptimizerRule was unsuccessful, but skipped failed rules is off, return error + (Err(e), None) => { + return Err(e.context(format!( + "Optimizer rule '{}' failed", + rule.name() + ))); } } } @@ -356,97 +414,22 @@ impl Optimizer { debug!("Optimizer took {} ms", start_time.elapsed().as_millis()); Ok(new_plan) } - - fn optimize_node( - &self, - rule: &Arc, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, - ) -> Result> { - // TODO: future feature: We can do Batch optimize - rule.try_optimize(plan, config) - } - - fn optimize_inputs( - &self, - rule: &Arc, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, - ) -> Result> { - let inputs = plan.inputs(); - let result = inputs - .iter() - .map(|sub_plan| self.optimize_recursively(rule, sub_plan, config)) - .collect::>>()?; - if result.is_empty() || result.iter().all(|o| o.is_none()) { - return Ok(None); - } - - let new_inputs = result - .into_iter() - .zip(inputs) - .map(|(new_plan, old_plan)| match new_plan { - Some(plan) => plan, - None => old_plan.clone(), - }) - .collect(); - - let exprs = plan.expressions(); - plan.with_new_exprs(exprs, new_inputs).map(Some) - } - - /// Use a rule to optimize the whole plan. - /// If the rule with `ApplyOrder`, we don't need to recursively handle children in rule. - pub fn optimize_recursively( - &self, - rule: &Arc, - plan: &LogicalPlan, - config: &dyn OptimizerConfig, - ) -> Result> { - match rule.apply_order() { - Some(order) => match order { - ApplyOrder::TopDown => { - let optimize_self_opt = self.optimize_node(rule, plan, config)?; - let optimize_inputs_opt = match &optimize_self_opt { - Some(optimized_plan) => { - self.optimize_inputs(rule, optimized_plan, config)? - } - _ => self.optimize_inputs(rule, plan, config)?, - }; - Ok(optimize_inputs_opt.or(optimize_self_opt)) - } - ApplyOrder::BottomUp => { - let optimize_inputs_opt = self.optimize_inputs(rule, plan, config)?; - let optimize_self_opt = match &optimize_inputs_opt { - Some(optimized_plan) => { - self.optimize_node(rule, optimized_plan, config)? - } - _ => self.optimize_node(rule, plan, config)?, - }; - Ok(optimize_self_opt.or(optimize_inputs_opt)) - } - }, - _ => rule.try_optimize(plan, config), - } - } } -/// Returns an error if plans have different schemas. +/// Returns an error if `new_plan`'s schema is different than `prev_schema` /// /// It ignores metadata and nullability. pub(crate) fn assert_schema_is_the_same( rule_name: &str, - prev_plan: &LogicalPlan, + prev_schema: &DFSchema, new_plan: &LogicalPlan, ) -> Result<()> { - let equivalent = new_plan - .schema() - .equivalent_names_and_types(prev_plan.schema()); + let equivalent = new_plan.schema().equivalent_names_and_types(prev_schema); if !equivalent { let e = DataFusionError::Internal(format!( "Failed due to a difference in schemas, original schema: {:?}, new schema: {:?}", - prev_plan.schema(), + prev_schema, new_plan.schema() )); Err(DataFusionError::Context( @@ -462,14 +445,15 @@ pub(crate) fn assert_schema_is_the_same( mod tests { use std::sync::{Arc, Mutex}; - use super::ApplyOrder; + use datafusion_common::{plan_err, DFSchema, DFSchemaRef, Result}; + use datafusion_expr::logical_plan::EmptyRelation; + use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, Projection}; + use crate::optimizer::Optimizer; use crate::test::test_table_scan; use crate::{OptimizerConfig, OptimizerContext, OptimizerRule}; - use datafusion_common::{plan_err, DFSchema, DFSchemaRef, Result}; - use datafusion_expr::logical_plan::EmptyRelation; - use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, Projection}; + use super::ApplyOrder; #[test] fn skip_failing_rule() { @@ -479,7 +463,7 @@ mod tests { produce_one_row: false, schema: Arc::new(DFSchema::empty()), }); - opt.optimize(&plan, &config, &observe).unwrap(); + opt.optimize(plan, &config, &observe).unwrap(); } #[test] @@ -490,7 +474,7 @@ mod tests { produce_one_row: false, schema: Arc::new(DFSchema::empty()), }); - let err = opt.optimize(&plan, &config, &observe).unwrap_err(); + let err = opt.optimize(plan, &config, &observe).unwrap_err(); assert_eq!( "Optimizer rule 'bad rule' failed\ncaused by\n\ Error during planning: rule failed", @@ -506,21 +490,27 @@ mod tests { produce_one_row: false, schema: Arc::new(DFSchema::empty()), }); - let err = opt.optimize(&plan, &config, &observe).unwrap_err(); + let err = opt.optimize(plan, &config, &observe).unwrap_err(); assert_eq!( - "Optimizer rule 'get table_scan rule' failed\ncaused by\nget table_scan rule\ncaused by\n\ - Internal error: Failed due to a difference in schemas, original schema: \ - DFSchema { inner: Schema { fields: \ - [Field { name: \"a\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, \ - Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, \ - Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {} }, \ - field_qualifiers: [Some(Bare { table: \"test\" }), Some(Bare { table: \"test\" }), Some(Bare { table: \"test\" })], \ - functional_dependencies: FunctionalDependencies { deps: [] } }, \ + "Optimizer rule 'get table_scan rule' failed\n\ + caused by\nget table_scan rule\ncaused by\n\ + Internal error: Failed due to a difference in schemas, \ + original schema: DFSchema { inner: Schema { \ + fields: [], \ + metadata: {} }, \ + field_qualifiers: [], \ + functional_dependencies: FunctionalDependencies { deps: [] } \ + }, \ new schema: DFSchema { inner: Schema { \ - fields: [], metadata: {} }, \ - field_qualifiers: [], \ - functional_dependencies: FunctionalDependencies { deps: [] } }.\n\ - This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker", + fields: [\ + Field { name: \"a\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, \ + Field { name: \"b\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, \ + Field { name: \"c\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }\ + ], \ + metadata: {} }, \ + field_qualifiers: [Some(Bare { table: \"test\" }), Some(Bare { table: \"test\" }), Some(Bare { table: \"test\" })], \ + functional_dependencies: FunctionalDependencies { deps: [] } }.\n\ + This was likely caused by a bug in DataFusion's code and we would welcome that you file an bug report in our issue tracker", err.strip_backtrace() ); } @@ -533,7 +523,7 @@ mod tests { produce_one_row: false, schema: Arc::new(DFSchema::empty()), }); - opt.optimize(&plan, &config, &observe).unwrap(); + opt.optimize(plan, &config, &observe).unwrap(); } #[test] @@ -554,7 +544,7 @@ mod tests { // optimizing should be ok, but the schema will have changed (no metadata) assert_ne!(plan.schema().as_ref(), input_schema.as_ref()); - let optimized_plan = opt.optimize(&plan, &config, &observe)?; + let optimized_plan = opt.optimize(plan, &config, &observe)?; // metadata was removed assert_eq!(optimized_plan.schema().as_ref(), input_schema.as_ref()); Ok(()) @@ -575,7 +565,7 @@ mod tests { let mut plans: Vec = Vec::new(); let final_plan = - opt.optimize(&initial_plan, &config, |p, _| plans.push(p.clone()))?; + opt.optimize(initial_plan.clone(), &config, |p, _| plans.push(p.clone()))?; // initial_plan is not observed, so we have 3 plans assert_eq!(3, plans.len()); @@ -601,7 +591,7 @@ mod tests { let mut plans: Vec = Vec::new(); let final_plan = - opt.optimize(&initial_plan, &config, |p, _| plans.push(p.clone()))?; + opt.optimize(initial_plan, &config, |p, _| plans.push(p.clone()))?; // initial_plan is not observed, so we have 4 plans assert_eq!(4, plans.len()); diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 2aca6f93254a..445109bbdf77 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -198,12 +198,12 @@ mod tests { use super::*; - fn assert_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_eq(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(PropagateEmptyRelation::new()), plan, expected) } fn assert_together_optimized_plan_eq( - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) -> Result<()> { assert_optimized_plan_eq_with_rules( @@ -226,7 +226,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_eq(&plan, expected) + assert_eq(plan, expected) } #[test] @@ -249,7 +249,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -262,7 +262,7 @@ mod tests { let plan = LogicalPlanBuilder::from(left).union(right)?.build()?; let expected = "TableScan: test"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -287,7 +287,7 @@ mod tests { let expected = "Union\ \n TableScan: test1\ \n TableScan: test4"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -312,7 +312,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -339,7 +339,7 @@ mod tests { let expected = "Union\ \n TableScan: test2\ \n TableScan: test3"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -352,7 +352,7 @@ mod tests { let plan = LogicalPlanBuilder::from(left).union(right)?.build()?; let expected = "TableScan: test"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -367,7 +367,7 @@ mod tests { .build()?; let expected = "EmptyRelation"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } #[test] @@ -400,6 +400,6 @@ mod tests { let expected = "Projection: a, b, c\ \n TableScan: test"; - assert_together_optimized_plan_eq(&plan, expected) + assert_together_optimized_plan_eq(plan, expected) } } diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index f3ce8bbcde72..2b123e3559f5 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -1028,11 +1028,11 @@ fn contain(e: &Expr, check_map: &HashMap) -> bool { #[cfg(test)] mod tests { + use super::*; use std::any::Any; use std::fmt::{Debug, Formatter}; use std::sync::Arc; - use super::*; use crate::optimizer::Optimizer; use crate::rewrite_disjunctive_predicate::RewriteDisjunctivePredicate; use crate::test::*; @@ -1040,6 +1040,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{DFSchema, DFSchemaRef, ScalarValue}; + use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ and, col, in_list, in_subquery, lit, logical_plan::JoinType, or, sum, BinaryExpr, @@ -1049,9 +1050,9 @@ mod tests { }; use async_trait::async_trait; - use datafusion_expr::expr::ScalarFunction; + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { crate::test::assert_optimized_plan_eq( Arc::new(PushDownFilter::new()), plan, @@ -1060,29 +1061,17 @@ mod tests { } fn assert_optimized_plan_eq_with_rewrite_predicate( - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) -> Result<()> { let optimizer = Optimizer::with_rules(vec![ Arc::new(RewriteDisjunctivePredicate::new()), Arc::new(PushDownFilter::new()), ]); - let mut optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - )? - .unwrap_or_else(|| plan.clone()); - optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.get(1).unwrap(), - &optimized_plan, - &OptimizerContext::new(), - )? - .unwrap_or_else(|| plan.clone()); + let optimized_plan = + optimizer.optimize(plan, &OptimizerContext::new(), observe)?; + let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(plan.schema(), optimized_plan.schema()); assert_eq!(expected, formatted_plan); Ok(()) } @@ -1098,7 +1087,7 @@ mod tests { let expected = "\ Projection: test.a, test.b\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1115,7 +1104,7 @@ mod tests { \n Limit: skip=0, fetch=10\ \n Projection: test.a, test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1125,7 +1114,7 @@ mod tests { .filter(lit(0i64).eq(lit(1i64)))? .build()?; let expected = "TableScan: test, full_filters=[Int64(0) = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1141,7 +1130,7 @@ mod tests { Projection: test.c, test.b\ \n Projection: test.a, test.b, test.c\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1155,7 +1144,7 @@ mod tests { let expected = "\ Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b) AS total_salary]]\ \n TableScan: test, full_filters=[test.a > Int64(10)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1168,7 +1157,7 @@ mod tests { let expected = "Filter: test.b > Int64(10)\ \n Aggregate: groupBy=[[test.b + test.a]], aggr=[[SUM(test.a), test.b]]\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1180,7 +1169,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.b + test.a]], aggr=[[SUM(test.a), test.b]]\ \n TableScan: test, full_filters=[test.b + test.a > Int64(10)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1195,7 +1184,7 @@ mod tests { Filter: b > Int64(10)\ \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b) AS b]]\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that a filter is pushed to before a projection, the filter expression is correctly re-written @@ -1210,7 +1199,7 @@ mod tests { let expected = "\ Projection: test.a AS b, test.c\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } fn add(left: Expr, right: Expr) -> Expr { @@ -1254,7 +1243,7 @@ mod tests { let expected = "\ Projection: test.a * Int32(2) + test.c AS b, test.c\ \n TableScan: test, full_filters=[test.a * Int32(2) + test.c = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that when a filter is pushed to after 2 projections, the filter expression is correctly re-written @@ -1286,7 +1275,7 @@ mod tests { Projection: b * Int32(3) AS a, test.c\ \n Projection: test.a * Int32(2) + test.c AS b, test.c\ \n TableScan: test, full_filters=[(test.a * Int32(2) + test.c) * Int32(3) = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[derive(Debug, PartialEq, Eq, Hash)] @@ -1349,7 +1338,7 @@ mod tests { let expected = "\ NoopPlan\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected)?; + assert_optimized_plan_eq(plan, expected)?; let custom_plan = LogicalPlan::Extension(Extension { node: Arc::new(NoopPlan { @@ -1366,7 +1355,7 @@ mod tests { Filter: test.c = Int64(2)\ \n NoopPlan\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected)?; + assert_optimized_plan_eq(plan, expected)?; let custom_plan = LogicalPlan::Extension(Extension { node: Arc::new(NoopPlan { @@ -1383,7 +1372,7 @@ mod tests { NoopPlan\ \n TableScan: test, full_filters=[test.a = Int64(1)]\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected)?; + assert_optimized_plan_eq(plan, expected)?; let custom_plan = LogicalPlan::Extension(Extension { node: Arc::new(NoopPlan { @@ -1401,7 +1390,7 @@ mod tests { \n NoopPlan\ \n TableScan: test, full_filters=[test.a = Int64(1)]\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that when two filters apply after an aggregation that only allows one to be pushed, one is pushed @@ -1434,7 +1423,7 @@ mod tests { \n Aggregate: groupBy=[[b]], aggr=[[SUM(test.c)]]\ \n Projection: test.a AS b, test.c\ \n TableScan: test, full_filters=[test.a > Int64(10)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that when a filter with two predicates is applied after an aggregation that only allows one to be pushed, one is pushed @@ -1468,7 +1457,7 @@ mod tests { \n Aggregate: groupBy=[[b]], aggr=[[SUM(test.c)]]\ \n Projection: test.a AS b, test.c\ \n TableScan: test, full_filters=[test.a > Int64(10)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that when two limits are in place, we jump neither @@ -1490,7 +1479,7 @@ mod tests { \n Limit: skip=0, fetch=20\ \n Projection: test.a, test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1505,7 +1494,7 @@ mod tests { let expected = "Union\ \n TableScan: test, full_filters=[test.a = Int64(1)]\ \n TableScan: test2, full_filters=[test2.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1528,7 +1517,7 @@ mod tests { \n SubqueryAlias: test2\ \n Projection: test.a AS b\ \n TableScan: test, full_filters=[test.a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1559,7 +1548,7 @@ mod tests { \n Projection: test1.d, test1.e, test1.f\ \n TableScan: test1, full_filters=[test1.d > Int32(2)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -1585,7 +1574,7 @@ mod tests { \n TableScan: test, full_filters=[test.a = Int32(1)]\ \n Projection: test1.a, test1.b, test1.c\ \n TableScan: test1, full_filters=[test1.a > Int32(2)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that filters with the same columns are correctly placed @@ -1619,7 +1608,7 @@ mod tests { \n Projection: test.a\ \n TableScan: test, full_filters=[test.a <= Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that filters to be placed on the same depth are ANDed @@ -1649,7 +1638,7 @@ mod tests { \n Limit: skip=0, fetch=1\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// verifies that filters on a plan with user nodes are not lost @@ -1675,7 +1664,7 @@ mod tests { TestUserDefined\ \n TableScan: test, full_filters=[test.a <= Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-on-join predicates on a column common to both sides is pushed to both sides @@ -1713,7 +1702,7 @@ mod tests { \n TableScan: test, full_filters=[test.a <= Int64(1)]\ \n Projection: test2.a\ \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-using-join predicates on a column common to both sides is pushed to both sides @@ -1750,7 +1739,7 @@ mod tests { \n TableScan: test, full_filters=[test.a <= Int64(1)]\ \n Projection: test2.a\ \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-join predicates with columns from both sides are converted to join filterss @@ -1792,7 +1781,7 @@ mod tests { \n TableScan: test\ \n Projection: test2.a, test2.b\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-join predicates with columns from one side of a join are pushed only to that side @@ -1834,7 +1823,7 @@ mod tests { \n TableScan: test, full_filters=[test.b <= Int64(1)]\ \n Projection: test2.a, test2.c\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-join predicates on the right side of a left join are not duplicated @@ -1873,7 +1862,7 @@ mod tests { \n TableScan: test\ \n Projection: test2.a\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-join predicates on the left side of a right join are not duplicated @@ -1911,7 +1900,7 @@ mod tests { \n TableScan: test\ \n Projection: test2.a\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-left-join predicate on a column common to both sides is only pushed to the left side @@ -1949,7 +1938,7 @@ mod tests { \n TableScan: test, full_filters=[test.a <= Int64(1)]\ \n Projection: test2.a\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// post-right-join predicate on a column common to both sides is only pushed to the right side @@ -1987,7 +1976,7 @@ mod tests { \n TableScan: test\ \n Projection: test2.a\ \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// single table predicate parts of ON condition should be pushed to both inputs @@ -2030,7 +2019,7 @@ mod tests { \n TableScan: test, full_filters=[test.c > UInt32(1)]\ \n Projection: test2.a, test2.b, test2.c\ \n TableScan: test2, full_filters=[test2.c > UInt32(4)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// join filter should be completely removed after pushdown @@ -2072,7 +2061,7 @@ mod tests { \n TableScan: test, full_filters=[test.b > UInt32(1)]\ \n Projection: test2.a, test2.b, test2.c\ \n TableScan: test2, full_filters=[test2.c > UInt32(4)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// predicate on join key in filter expression should be pushed down to both inputs @@ -2112,7 +2101,7 @@ mod tests { \n TableScan: test, full_filters=[test.a > UInt32(1)]\ \n Projection: test2.b\ \n TableScan: test2, full_filters=[test2.b > UInt32(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// single table predicate parts of ON condition should be pushed to right input @@ -2155,7 +2144,7 @@ mod tests { \n TableScan: test\ \n Projection: test2.a, test2.b, test2.c\ \n TableScan: test2, full_filters=[test2.c > UInt32(4)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// single table predicate parts of ON condition should be pushed to left input @@ -2198,7 +2187,7 @@ mod tests { \n TableScan: test, full_filters=[test.a > UInt32(1)]\ \n Projection: test2.a, test2.b, test2.c\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// single table predicate parts of ON condition should not be pushed @@ -2236,7 +2225,7 @@ mod tests { ); let expected = &format!("{plan:?}"); - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } struct PushDownProvider { @@ -2295,7 +2284,7 @@ mod tests { let expected = "\ TableScan: test, full_filters=[a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2306,7 +2295,7 @@ mod tests { let expected = "\ Filter: a = Int64(1)\ \n TableScan: test, partial_filters=[a = Int64(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2314,7 +2303,7 @@ mod tests { let plan = table_scan_with_pushdown_provider(TableProviderFilterPushDown::Inexact)?; - let optimised_plan = PushDownFilter::new() + let optimized_plan = PushDownFilter::new() .try_optimize(&plan, &OptimizerContext::new()) .expect("failed to optimize plan") .unwrap(); @@ -2325,7 +2314,7 @@ mod tests { // Optimizing the same plan multiple times should produce the same plan // each time. - assert_optimized_plan_eq(&optimised_plan, expected) + assert_optimized_plan_eq(optimized_plan, expected) } #[test] @@ -2336,7 +2325,7 @@ mod tests { let expected = "\ Filter: a = Int64(1)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2365,7 +2354,7 @@ mod tests { \n Filter: a = Int64(10) AND b > Int64(11)\ \n TableScan: test projection=[a], partial_filters=[a = Int64(10), b > Int64(11)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2396,7 +2385,7 @@ Projection: a, b "# .trim(); - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2424,7 +2413,7 @@ Projection: a, b \n TableScan: test, full_filters=[test.a > Int64(10), test.c > Int64(10)]\ "; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2456,7 +2445,7 @@ Projection: a, b \n TableScan: test, full_filters=[test.a > Int64(10), test.c > Int64(10)]\ "; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2481,7 +2470,7 @@ Projection: a, b Projection: test.a AS b, test.c AS d\ \n TableScan: test, full_filters=[test.a > Int64(10), test.c > Int64(10)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// predicate on join key in filter expression should be pushed down to both inputs @@ -2521,7 +2510,7 @@ Projection: a, b \n TableScan: test, full_filters=[test.a > UInt32(1)]\ \n Projection: test2.b AS d\ \n TableScan: test2, full_filters=[test2.b > UInt32(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2550,7 +2539,7 @@ Projection: a, b Projection: test.a AS b, test.c\ \n TableScan: test, full_filters=[test.a IN ([UInt32(1), UInt32(2), UInt32(3), UInt32(4)])]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2582,7 +2571,7 @@ Projection: a, b \n Projection: test.a AS b, test.c\ \n TableScan: test, full_filters=[test.a IN ([UInt32(1), UInt32(2), UInt32(3), UInt32(4)])]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2618,7 +2607,7 @@ Projection: a, b \n Subquery:\ \n Projection: sq.c\ \n TableScan: sq"; - assert_optimized_plan_eq(&plan, expected_after) + assert_optimized_plan_eq(plan, expected_after) } #[test] @@ -2651,7 +2640,7 @@ Projection: a, b \n Projection: Int64(0) AS a\ \n Filter: Int64(0) = Int64(1)\ \n EmptyRelation"; - assert_optimized_plan_eq(&plan, expected_after) + assert_optimized_plan_eq(plan, expected_after) } #[test] @@ -2679,14 +2668,14 @@ Projection: a, b \n TableScan: test, full_filters=[test.b > UInt32(1) OR test.c < UInt32(10)]\ \n Projection: test1.a AS d, test1.a AS e\ \n TableScan: test1"; - assert_optimized_plan_eq_with_rewrite_predicate(&plan, expected)?; + assert_optimized_plan_eq_with_rewrite_predicate(plan.clone(), expected)?; // Originally global state which can help to avoid duplicate Filters been generated and pushed down. // Now the global state is removed. Need to double confirm that avoid duplicate Filters. let optimized_plan = PushDownFilter::new() .try_optimize(&plan, &OptimizerContext::new())? .expect("failed to optimize plan"); - assert_optimized_plan_eq(&optimized_plan, expected) + assert_optimized_plan_eq(optimized_plan, expected) } #[test] @@ -2727,7 +2716,7 @@ Projection: a, b \n TableScan: test1, full_filters=[test1.b > UInt32(1)]\ \n Projection: test2.a, test2.b\ \n TableScan: test2, full_filters=[test2.b > UInt32(2)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2768,7 +2757,7 @@ Projection: a, b \n TableScan: test1, full_filters=[test1.b > UInt32(1)]\ \n Projection: test2.a, test2.b\ \n TableScan: test2, full_filters=[test2.b > UInt32(2)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2814,7 +2803,7 @@ Projection: a, b \n TableScan: test1\ \n Projection: test2.a, test2.b\ \n TableScan: test2, full_filters=[test2.b > UInt32(2)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -2859,7 +2848,7 @@ Projection: a, b \n TableScan: test1, full_filters=[test1.b > UInt32(1)]\ \n Projection: test2.a, test2.b\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[derive(Debug)] @@ -2919,7 +2908,7 @@ Projection: a, b \n Projection: test1.a, SUM(test1.b), TestScalarUDF() + Int32(1) AS r\ \n Aggregate: groupBy=[[test1.a]], aggr=[[SUM(test1.b)]]\ \n TableScan: test1, full_filters=[test1.a > Int32(5)]"; - assert_optimized_plan_eq(&plan, expected_after) + assert_optimized_plan_eq(plan, expected_after) } #[test] @@ -2965,6 +2954,6 @@ Projection: a, b \n Inner Join: test1.a = test2.a\ \n TableScan: test1\ \n TableScan: test2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } } diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index cca6c3fd9bd1..6f1d7bf97cfe 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -285,7 +285,7 @@ mod test { max, }; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(PushDownLimit::new()), plan, expected) } @@ -304,7 +304,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -322,7 +322,7 @@ mod test { let expected = "Limit: skip=0, fetch=10\ \n TableScan: test, fetch=10"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -339,7 +339,7 @@ mod test { \n Aggregate: groupBy=[[test.a]], aggr=[[MAX(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -359,7 +359,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -376,7 +376,7 @@ mod test { \n Sort: test.a, fetch=10\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -393,7 +393,7 @@ mod test { \n Sort: test.a, fetch=15\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -412,7 +412,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -427,7 +427,7 @@ mod test { let expected = "Limit: skip=10, fetch=None\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -445,7 +445,7 @@ mod test { \n Limit: skip=10, fetch=1000\ \n TableScan: test, fetch=1010"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -462,7 +462,7 @@ mod test { \n Limit: skip=10, fetch=990\ \n TableScan: test, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -479,7 +479,7 @@ mod test { \n Limit: skip=10, fetch=1000\ \n TableScan: test, fetch=1010"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -495,7 +495,7 @@ mod test { let expected = "Limit: skip=10, fetch=10\ \n TableScan: test, fetch=20"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -512,7 +512,7 @@ mod test { \n Aggregate: groupBy=[[test.a]], aggr=[[MAX(test.b)]]\ \n TableScan: test"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -532,7 +532,7 @@ mod test { \n Limit: skip=0, fetch=1010\ \n TableScan: test, fetch=1010"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -556,7 +556,7 @@ mod test { \n TableScan: test\ \n TableScan: test2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -580,7 +580,7 @@ mod test { \n TableScan: test\ \n TableScan: test2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -609,7 +609,7 @@ mod test { \n Projection: test2.a\ \n TableScan: test2"; - assert_optimized_plan_equal(&outer_query, expected) + assert_optimized_plan_equal(outer_query, expected) } #[test] @@ -638,7 +638,7 @@ mod test { \n Projection: test2.a\ \n TableScan: test2"; - assert_optimized_plan_equal(&outer_query, expected) + assert_optimized_plan_equal(outer_query, expected) } #[test] @@ -664,7 +664,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected)?; + assert_optimized_plan_equal(plan, expected)?; let plan = LogicalPlanBuilder::from(table_scan_1.clone()) .join( @@ -683,7 +683,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected)?; + assert_optimized_plan_equal(plan, expected)?; let plan = LogicalPlanBuilder::from(table_scan_1.clone()) .join( @@ -702,7 +702,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected)?; + assert_optimized_plan_equal(plan, expected)?; let plan = LogicalPlanBuilder::from(table_scan_1.clone()) .join( @@ -720,7 +720,7 @@ mod test { \n TableScan: test, fetch=1000\ \n TableScan: test2"; - assert_optimized_plan_equal(&plan, expected)?; + assert_optimized_plan_equal(plan, expected)?; let plan = LogicalPlanBuilder::from(table_scan_1.clone()) .join( @@ -738,7 +738,7 @@ mod test { \n TableScan: test, fetch=1000\ \n TableScan: test2"; - assert_optimized_plan_equal(&plan, expected)?; + assert_optimized_plan_equal(plan, expected)?; let plan = LogicalPlanBuilder::from(table_scan_1.clone()) .join( @@ -756,7 +756,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected)?; + assert_optimized_plan_equal(plan, expected)?; let plan = LogicalPlanBuilder::from(table_scan_1) .join( @@ -774,7 +774,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -799,7 +799,7 @@ mod test { \n TableScan: test, fetch=1000\ \n TableScan: test2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -824,7 +824,7 @@ mod test { \n TableScan: test, fetch=1010\ \n TableScan: test2"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -849,7 +849,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -874,7 +874,7 @@ mod test { \n Limit: skip=0, fetch=1010\ \n TableScan: test2, fetch=1010"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -894,7 +894,7 @@ mod test { \n Limit: skip=0, fetch=1000\ \n TableScan: test2, fetch=1000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -914,7 +914,7 @@ mod test { \n Limit: skip=0, fetch=2000\ \n TableScan: test2, fetch=2000"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -929,7 +929,7 @@ mod test { let expected = "Limit: skip=1000, fetch=0\ \n TableScan: test, fetch=0"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -944,7 +944,7 @@ mod test { let expected = "Limit: skip=1000, fetch=0\ \n TableScan: test, fetch=0"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -961,6 +961,6 @@ mod test { \n Limit: skip=1000, fetch=0\ \n TableScan: test, fetch=0"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index ae57ed9e5a34..2f578094b3bc 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -24,7 +24,7 @@ mod tests { use crate::optimize_projections::OptimizeProjections; use crate::optimizer::Optimizer; use crate::test::*; - use crate::OptimizerContext; + use crate::{OptimizerContext, OptimizerRule}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{Column, DFSchema, Result}; use datafusion_expr::builder::table_scan_with_filters; @@ -48,7 +48,7 @@ mod tests { let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ \n TableScan: test projection=[b]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -62,7 +62,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.b)]]\ \n TableScan: test projection=[b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -78,7 +78,7 @@ mod tests { \n SubqueryAlias: a\ \n TableScan: test projection=[b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -95,7 +95,7 @@ mod tests { \n Filter: test.c > Int32(1)\ \n TableScan: test projection=[b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -120,7 +120,7 @@ mod tests { Aggregate: groupBy=[[]], aggr=[[MAX(m4.tag.one) AS tag.one]]\ \n TableScan: m4 projection=[tag.one]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -134,7 +134,7 @@ mod tests { let expected = "Projection: test.a, test.c, test.b\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -144,7 +144,7 @@ mod tests { let plan = table_scan(Some("test"), &schema, Some(vec![1, 0, 2]))?.build()?; let expected = "TableScan: test projection=[b, a, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -157,7 +157,7 @@ mod tests { let expected = "Projection: test.a, test.b\ \n TableScan: test projection=[b, a]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -170,7 +170,7 @@ mod tests { let expected = "Projection: test.c, test.b, test.a\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -192,7 +192,7 @@ mod tests { \n Filter: test.c > Int32(1)\ \n Projection: test.c, test.b, test.a\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -212,7 +212,7 @@ mod tests { \n TableScan: test projection=[a, b]\ \n TableScan: test2 projection=[c1]"; - let optimized_plan = optimize(&plan)?; + let optimized_plan = optimize(plan)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); @@ -264,7 +264,7 @@ mod tests { \n TableScan: test projection=[a, b]\ \n TableScan: test2 projection=[c1]"; - let optimized_plan = optimize(&plan)?; + let optimized_plan = optimize(plan)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); @@ -314,7 +314,7 @@ mod tests { \n TableScan: test projection=[a, b]\ \n TableScan: test2 projection=[a]"; - let optimized_plan = optimize(&plan)?; + let optimized_plan = optimize(plan)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); @@ -358,7 +358,7 @@ mod tests { let expected = "Projection: CAST(test.c AS Float64)\ \n TableScan: test projection=[c]"; - assert_optimized_plan_eq(&projection, expected) + assert_optimized_plan_eq(projection, expected) } #[test] @@ -374,7 +374,7 @@ mod tests { let expected = "TableScan: test projection=[a, b]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -395,7 +395,7 @@ mod tests { let expected = "TableScan: test projection=[a, b]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -415,7 +415,7 @@ mod tests { \n Projection: test.c, test.a\ \n TableScan: test projection=[a, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -424,7 +424,7 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan).build()?; // should expand projection to all columns without projection let expected = "TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -435,7 +435,7 @@ mod tests { .build()?; let expected = "Projection: Int64(1), Int64(2)\ \n TableScan: test projection=[]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// tests that it removes unused columns in projections @@ -454,14 +454,14 @@ mod tests { assert_fields_eq(&plan, vec!["c", "MAX(test.a)"]); - let plan = optimize(&plan).expect("failed to optimize plan"); + let plan = optimize(plan).expect("failed to optimize plan"); let expected = "\ Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.a)]]\ \n Filter: test.c > Int32(1)\ \n Projection: test.c, test.a\ \n TableScan: test projection=[a, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// tests that it removes un-needed projections @@ -483,7 +483,7 @@ mod tests { Projection: Int32(1) AS a\ \n TableScan: test projection=[]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -512,7 +512,7 @@ mod tests { Projection: Int32(1) AS a\ \n TableScan: test projection=[], full_filters=[b = Int32(1)]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } /// tests that optimizing twice yields same plan @@ -525,9 +525,9 @@ mod tests { .project(vec![lit(1).alias("a")])? .build()?; - let optimized_plan1 = optimize(&plan).expect("failed to optimize plan"); + let optimized_plan1 = optimize(plan).expect("failed to optimize plan"); let optimized_plan2 = - optimize(&optimized_plan1).expect("failed to optimize plan"); + optimize(optimized_plan1.clone()).expect("failed to optimize plan"); let formatted_plan1 = format!("{optimized_plan1:?}"); let formatted_plan2 = format!("{optimized_plan2:?}"); @@ -556,7 +556,7 @@ mod tests { \n Aggregate: groupBy=[[test.a, test.c]], aggr=[[MAX(test.b)]]\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -582,7 +582,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.b), COUNT(test.b) FILTER (WHERE test.c > Int32(42)) AS count2]]\ \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -599,7 +599,7 @@ mod tests { \n Distinct:\ \n TableScan: test projection=[a, b]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -638,25 +638,23 @@ mod tests { \n WindowAggr: windowExpr=[[MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: test projection=[a, b]"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { let optimized_plan = optimize(plan).expect("failed to optimize plan"); let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); Ok(()) } - fn optimize(plan: &LogicalPlan) -> Result { + fn optimize(plan: LogicalPlan) -> Result { let optimizer = Optimizer::with_rules(vec![Arc::new(OptimizeProjections::new())]); - let optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - )? - .unwrap_or_else(|| plan.clone()); + let optimized_plan = + optimizer.optimize(plan, &OptimizerContext::new(), observe)?; + Ok(optimized_plan) } + + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} } diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index 752915be69c0..f464506057ff 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -172,7 +172,7 @@ mod tests { assert_optimized_plan_eq( Arc::new(ReplaceDistinctWithAggregate::new()), - &plan, + plan, expected, ) } @@ -195,7 +195,7 @@ mod tests { assert_optimized_plan_eq( Arc::new(ReplaceDistinctWithAggregate::new()), - &plan, + plan, expected, ) } diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index a2c4eabcaae6..a8999f9c1d3c 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -429,7 +429,7 @@ mod tests { \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -485,7 +485,7 @@ mod tests { \n TableScan: lineitem [l_orderkey:Int64, l_partkey:Int64, l_suppkey:Int64, l_linenumber:Int32, l_quantity:Float64, l_extendedprice:Float64]"; assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -523,7 +523,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -559,7 +559,7 @@ mod tests { \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -593,7 +593,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -732,7 +732,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -798,7 +798,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -837,7 +837,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -877,7 +877,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -910,7 +910,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -942,7 +942,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -973,7 +973,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -1030,7 +1030,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) @@ -1079,7 +1079,7 @@ mod tests { assert_multi_rules_optimized_plan_eq_display_indent( vec![Arc::new(ScalarSubqueryToJoin::new())], - &plan, + plan, expected, ); Ok(()) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 076bf4e24296..602994a9e3e2 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -313,7 +313,7 @@ mod tests { min, sum, AggregateFunction, }; - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq_display_indent( Arc::new(SingleDistinctToGroupBy::new()), plan, @@ -335,7 +335,7 @@ mod tests { "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]] [MAX(test.b):UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -352,7 +352,7 @@ mod tests { \n Aggregate: groupBy=[[test.b AS alias1]], aggr=[[]] [alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Currently this optimization is disabled for CUBE/ROLLUP/GROUPING SET @@ -373,7 +373,7 @@ mod tests { let expected = "Aggregate: groupBy=[[GROUPING SETS ((test.a), (test.b))]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Currently this optimization is disabled for CUBE/ROLLUP/GROUPING SET @@ -391,7 +391,7 @@ mod tests { let expected = "Aggregate: groupBy=[[CUBE (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } // Currently this optimization is disabled for CUBE/ROLLUP/GROUPING SET @@ -410,7 +410,7 @@ mod tests { let expected = "Aggregate: groupBy=[[ROLLUP (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -426,7 +426,7 @@ mod tests { \n Aggregate: groupBy=[[Int32(2) * test.b AS alias1]], aggr=[[]] [alias1:Int32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -443,7 +443,7 @@ mod tests { \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -461,7 +461,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(DISTINCT test.b), COUNT(DISTINCT test.c)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -490,7 +490,7 @@ mod tests { \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -508,7 +508,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(DISTINCT test.b), COUNT(test.c)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N, COUNT(test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -525,7 +525,7 @@ mod tests { \n Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -555,7 +555,7 @@ mod tests { \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[SUM(test.c) AS alias2]] [a:UInt32, alias1:UInt32, alias2:UInt64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -574,7 +574,7 @@ mod tests { \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[SUM(test.c) AS alias2, MAX(test.c) AS alias3]] [a:UInt32, alias1:UInt32, alias2:UInt64;N, alias3:UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -593,7 +593,7 @@ mod tests { \n Aggregate: groupBy=[[test.c, test.b AS alias1]], aggr=[[MIN(test.a) AS alias2]] [c:UInt32, alias1:UInt32, alias2:UInt32;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -616,7 +616,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a) FILTER (WHERE test.a > Int32(5)), COUNT(DISTINCT test.b)]] [c:UInt32, SUM(test.a) FILTER (WHERE test.a > Int32(5)):UInt64;N, COUNT(DISTINCT test.b):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -639,7 +639,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a), COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5))]] [c:UInt32, SUM(test.a):UInt64;N, COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -662,7 +662,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a) ORDER BY [test.a], COUNT(DISTINCT test.b)]] [c:UInt32, SUM(test.a) ORDER BY [test.a]:UInt64;N, COUNT(DISTINCT test.b):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -685,7 +685,7 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a), COUNT(DISTINCT test.a) ORDER BY [test.a]]] [c:UInt32, SUM(test.a):UInt64;N, COUNT(DISTINCT test.a) ORDER BY [test.a]:Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } #[test] @@ -708,6 +708,6 @@ mod tests { let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a), COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a]]] [c:UInt32, SUM(test.a):UInt64;N, COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a]:Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; - assert_optimized_plan_equal(&plan, expected) + assert_optimized_plan_equal(plan, expected) } } diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index e691fe9a5351..cafda8359aa3 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -16,7 +16,7 @@ // under the License. use crate::analyzer::{Analyzer, AnalyzerRule}; -use crate::optimizer::{assert_schema_is_the_same, Optimizer}; +use crate::optimizer::Optimizer; use crate::{OptimizerContext, OptimizerRule}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::config::ConfigOptions; @@ -150,22 +150,19 @@ pub fn assert_analyzer_check_err( } } } + +fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + pub fn assert_optimized_plan_eq( rule: Arc, - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) -> Result<()> { - let optimizer = Optimizer::with_rules(vec![rule.clone()]); - let optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - )? - .unwrap_or_else(|| plan.clone()); + // Apply the rule once + let opt_context = OptimizerContext::new().with_max_passes(1); - // Ensure schemas always match after an optimization - assert_schema_is_the_same(rule.name(), plan, &optimized_plan)?; + let optimizer = Optimizer::with_rules(vec![rule.clone()]); + let optimized_plan = optimizer.optimize(plan, &opt_context, observe)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); @@ -174,7 +171,7 @@ pub fn assert_optimized_plan_eq( pub fn assert_optimized_plan_eq_with_rules( rules: Vec>, - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) -> Result<()> { fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} @@ -187,58 +184,44 @@ pub fn assert_optimized_plan_eq_with_rules( .expect("failed to optimize plan"); let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); - assert_eq!(plan.schema(), optimized_plan.schema()); Ok(()) } pub fn assert_optimized_plan_eq_display_indent( rule: Arc, - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) { let optimizer = Optimizer::with_rules(vec![rule]); let optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - ) - .expect("failed to optimize plan") - .unwrap_or_else(|| plan.clone()); + .optimize(plan, &OptimizerContext::new(), observe) + .expect("failed to optimize plan"); let formatted_plan = optimized_plan.display_indent_schema().to_string(); assert_eq!(formatted_plan, expected); } pub fn assert_multi_rules_optimized_plan_eq_display_indent( rules: Vec>, - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) { let optimizer = Optimizer::with_rules(rules); - let mut optimized_plan = plan.clone(); - for rule in &optimizer.rules { - optimized_plan = optimizer - .optimize_recursively(rule, &optimized_plan, &OptimizerContext::new()) - .expect("failed to optimize plan") - .unwrap_or_else(|| optimized_plan.clone()); - } + let optimized_plan = optimizer + .optimize(plan, &OptimizerContext::new(), observe) + .expect("failed to optimize plan"); let formatted_plan = optimized_plan.display_indent_schema().to_string(); assert_eq!(formatted_plan, expected); } pub fn assert_optimizer_err( rule: Arc, - plan: &LogicalPlan, + plan: LogicalPlan, expected: &str, ) { let optimizer = Optimizer::with_rules(vec![rule]); - let res = optimizer.optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - ); + let res = optimizer.optimize(plan, &OptimizerContext::new(), observe); match res { - Ok(plan) => assert_eq!(format!("{}", plan.unwrap().display_indent()), "An error"), + Ok(plan) => assert_eq!(format!("{}", plan.display_indent()), "An error"), Err(ref e) => { let actual = format!("{e}"); if expected.is_empty() || !actual.contains(expected) { @@ -250,16 +233,11 @@ pub fn assert_optimizer_err( pub fn assert_optimization_skipped( rule: Arc, - plan: &LogicalPlan, + plan: LogicalPlan, ) -> Result<()> { let optimizer = Optimizer::with_rules(vec![rule]); - let new_plan = optimizer - .optimize_recursively( - optimizer.rules.first().unwrap(), - plan, - &OptimizerContext::new(), - )? - .unwrap_or_else(|| plan.clone()); + let new_plan = optimizer.optimize(plan.clone(), &OptimizerContext::new(), observe)?; + assert_eq!( format!("{}", plan.display_indent()), format!("{}", new_plan.display_indent()) diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 61d2535930b2..01db5e817c56 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -25,7 +25,7 @@ use datafusion_common::{plan_err, Result}; use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF}; use datafusion_optimizer::analyzer::Analyzer; use datafusion_optimizer::optimizer::Optimizer; -use datafusion_optimizer::{OptimizerConfig, OptimizerContext}; +use datafusion_optimizer::{OptimizerConfig, OptimizerContext, OptimizerRule}; use datafusion_sql::planner::{ContextProvider, SqlToRel}; use datafusion_sql::sqlparser::ast::Statement; use datafusion_sql::sqlparser::dialect::GenericDialect; @@ -315,9 +315,11 @@ fn test_sql(sql: &str) -> Result { let optimizer = Optimizer::new(); // analyze and optimize the logical plan let plan = analyzer.execute_and_check(&plan, config.options(), |_, _| {})?; - optimizer.optimize(&plan, &config, |_, _| {}) + optimizer.optimize(plan, &config, observe) } +fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + #[derive(Default)] struct MyContextProvider { options: ConfigOptions, diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index da9b4168e7e0..135ab8075425 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -587,7 +587,7 @@ FROM t1 ---- 11 11 11 -# subsequent inner join +# subsequent inner join query III rowsort SELECT t1.t1_id, t2.t2_id, t3.t3_id FROM t1 From 1ec65a4a4a697d382d64ac2382b8486709dcf680 Mon Sep 17 00:00:00 2001 From: "D.B. Schwartz" <1689014+cisaacson@users.noreply.github.com> Date: Wed, 10 Apr 2024 10:00:47 -0600 Subject: [PATCH 05/39] Further clarification of the supports_filters_pushdown documentation (#9988) * Further refinement of the comment * Add code example example of how to support a filter * Update supports_filters_pushdown example so that it compiles * Add comments to example code in supports_filters_pushdown doc * Change example to use functional style * Fixed several issues with the supports_filters_pushdown doc; still need all required TableProvider impl fns * cargo fmt * Update example so it compiles and add headings * clean * remove to_string() --------- Co-authored-by: Andrew Lamb --- datafusion/core/src/datasource/provider.rs | 77 ++++++++++++++++++++-- 1 file changed, 70 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 9aac072ed4e2..100011952b3b 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -161,20 +161,83 @@ pub trait TableProvider: Sync + Send { /// Specify if DataFusion should provide filter expressions to the /// TableProvider to apply *during* the scan. /// - /// The return value must have one element for each filter expression passed - /// in. The value of each element indicates if the TableProvider can apply - /// that particular filter during the scan. - /// /// Some TableProviders can evaluate filters more efficiently than the /// `Filter` operator in DataFusion, for example by using an index. /// - /// By default, returns [`Unsupported`] for all filters, meaning no filters - /// will be provided to [`Self::scan`]. If the TableProvider can implement - /// filter pushdown, it should return either [`Exact`] or [`Inexact`]. + /// # Parameters and Return Value + /// + /// The return `Vec` must have one element for each element of the `filters` + /// argument. The value of each element indicates if the TableProvider can + /// apply the corresponding filter during the scan. The position in the return + /// value corresponds to the expression in the `filters` parameter. + /// + /// If the length of the resulting `Vec` does not match the `filters` input + /// an error will be thrown. + /// + /// Each element in the resulting `Vec` is one of the following: + /// * [`Exact`] or [`Inexact`]: The TableProvider can apply the filter + /// during scan + /// * [`Unsupported`]: The TableProvider cannot apply the filter during scan + /// + /// By default, this function returns [`Unsupported`] for all filters, + /// meaning no filters will be provided to [`Self::scan`]. /// /// [`Unsupported`]: TableProviderFilterPushDown::Unsupported /// [`Exact`]: TableProviderFilterPushDown::Exact /// [`Inexact`]: TableProviderFilterPushDown::Inexact + /// # Example + /// + /// ```rust + /// # use std::any::Any; + /// # use std::sync::Arc; + /// # use arrow_schema::SchemaRef; + /// # use async_trait::async_trait; + /// # use datafusion::datasource::TableProvider; + /// # use datafusion::error::{Result, DataFusionError}; + /// # use datafusion::execution::context::SessionState; + /// # use datafusion_expr::{Expr, TableProviderFilterPushDown, TableType}; + /// # use datafusion_physical_plan::ExecutionPlan; + /// // Define a struct that implements the TableProvider trait + /// struct TestDataSource {} + /// + /// #[async_trait] + /// impl TableProvider for TestDataSource { + /// # fn as_any(&self) -> &dyn Any { todo!() } + /// # fn schema(&self) -> SchemaRef { todo!() } + /// # fn table_type(&self) -> TableType { todo!() } + /// # async fn scan(&self, s: &SessionState, p: Option<&Vec>, f: &[Expr], l: Option) -> Result> { + /// todo!() + /// # } + /// // Override the supports_filters_pushdown to evaluate which expressions + /// // to accept as pushdown predicates. + /// fn supports_filters_pushdown(&self, filters: &[&Expr]) -> Result> { + /// // Process each filter + /// let support: Vec<_> = filters.iter().map(|expr| { + /// match expr { + /// // This example only supports a between expr with a single column named "c1". + /// Expr::Between(between_expr) => { + /// between_expr.expr + /// .try_into_col() + /// .map(|column| { + /// if column.name == "c1" { + /// TableProviderFilterPushDown::Exact + /// } else { + /// TableProviderFilterPushDown::Unsupported + /// } + /// }) + /// // If there is no column in the expr set the filter to unsupported. + /// .unwrap_or(TableProviderFilterPushDown::Unsupported) + /// } + /// _ => { + /// // For all other cases return Unsupported. + /// TableProviderFilterPushDown::Unsupported + /// } + /// } + /// }).collect(); + /// Ok(support) + /// } + /// } + /// ``` fn supports_filters_pushdown( &self, filters: &[&Expr], From ed374678d1beac56d39e673eb0edb78f34458f68 Mon Sep 17 00:00:00 2001 From: Yang Jiang Date: Thu, 11 Apr 2024 00:48:49 +0800 Subject: [PATCH 06/39] Prune columns are all null in ParquetExec by row_counts , handle IS NOT NULL (#9989) * Prune columns are all null in ParquetExec by row_counts in pruning statistics * fix clippy * Update datafusion/core/tests/parquet/row_group_pruning.rs Co-authored-by: Ruihang Xia * fix comment and support isNotNUll * add test * fix conflict --------- Co-authored-by: Ruihang Xia --- .../physical_plan/parquet/row_groups.rs | 10 +++- .../core/src/physical_optimizer/pruning.rs | 38 +++++++++--- datafusion/core/tests/parquet/mod.rs | 30 ++++++++++ .../core/tests/parquet/row_group_pruning.rs | 60 +++++++++++++++++++ 4 files changed, 128 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 6600dd07d7fd..2b9665954842 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -338,8 +338,10 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { scalar.to_array().ok() } - fn row_counts(&self, _column: &Column) -> Option { - None + fn row_counts(&self, column: &Column) -> Option { + let (c, _) = self.column(&column.name)?; + let scalar = ScalarValue::UInt64(Some(c.num_values() as u64)); + scalar.to_array().ok() } fn contained( @@ -1022,15 +1024,17 @@ mod tests { column_statistics: Vec, ) -> RowGroupMetaData { let mut columns = vec![]; + let number_row = 1000; for (i, s) in column_statistics.iter().enumerate() { let column = ColumnChunkMetaData::builder(schema_descr.column(i)) .set_statistics(s.clone()) + .set_num_values(number_row) .build() .unwrap(); columns.push(column); } RowGroupMetaData::builder(schema_descr.clone()) - .set_num_rows(1000) + .set_num_rows(number_row) .set_total_byte_size(2000) .set_column_metadata(columns) .build() diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index dc7e0529decb..ebb811408fb3 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -335,6 +335,7 @@ pub trait PruningStatistics { /// `x < 5` | `CASE WHEN x_null_count = x_row_count THEN false ELSE x_max < 5 END` /// `x = 5 AND y = 10` | `CASE WHEN x_null_count = x_row_count THEN false ELSE x_min <= 5 AND 5 <= x_max END AND CASE WHEN y_null_count = y_row_count THEN false ELSE y_min <= 10 AND 10 <= y_max END` /// `x IS NULL` | `x_null_count > 0` +/// `x IS NOT NULL` | `x_null_count = 0` /// `CAST(x as int) = 5` | `CASE WHEN x_null_count = x_row_count THEN false ELSE CAST(x_min as int) <= 5 AND 5 <= CAST(x_max as int) END` /// /// ## Predicate Evaluation @@ -1239,10 +1240,15 @@ fn build_single_column_expr( /// returns a pruning expression in terms of IsNull that will evaluate to true /// if the column may contain null, and false if definitely does not /// contain null. +/// If set `with_not` to true: which means is not null +/// Given an expression reference to `expr`, if `expr` is a column expression, +/// returns a pruning expression in terms of IsNotNull that will evaluate to true +/// if the column not contain any null, and false if definitely contain null. fn build_is_null_column_expr( expr: &Arc, schema: &Schema, required_columns: &mut RequiredColumns, + with_not: bool, ) -> Option> { if let Some(col) = expr.as_any().downcast_ref::() { let field = schema.field_with_name(col.name()).ok()?; @@ -1251,12 +1257,21 @@ fn build_is_null_column_expr( required_columns .null_count_column_expr(col, expr, null_count_field) .map(|null_count_column_expr| { - // IsNull(column) => null_count > 0 - Arc::new(phys_expr::BinaryExpr::new( - null_count_column_expr, - Operator::Gt, - Arc::new(phys_expr::Literal::new(ScalarValue::UInt64(Some(0)))), - )) as _ + if with_not { + // IsNotNull(column) => null_count = 0 + Arc::new(phys_expr::BinaryExpr::new( + null_count_column_expr, + Operator::Eq, + Arc::new(phys_expr::Literal::new(ScalarValue::UInt64(Some(0)))), + )) as _ + } else { + // IsNull(column) => null_count > 0 + Arc::new(phys_expr::BinaryExpr::new( + null_count_column_expr, + Operator::Gt, + Arc::new(phys_expr::Literal::new(ScalarValue::UInt64(Some(0)))), + )) as _ + } }) .ok() } else { @@ -1287,9 +1302,18 @@ fn build_predicate_expression( // predicate expression can only be a binary expression let expr_any = expr.as_any(); if let Some(is_null) = expr_any.downcast_ref::() { - return build_is_null_column_expr(is_null.arg(), schema, required_columns) + return build_is_null_column_expr(is_null.arg(), schema, required_columns, false) .unwrap_or(unhandled); } + if let Some(is_not_null) = expr_any.downcast_ref::() { + return build_is_null_column_expr( + is_not_null.arg(), + schema, + required_columns, + true, + ) + .unwrap_or(unhandled); + } if let Some(col) = expr_any.downcast_ref::() { return build_single_column_expr(col, schema, required_columns, false) .unwrap_or(unhandled); diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index b4415d638ada..f36afe1976b1 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -28,6 +28,7 @@ use arrow::{ record_batch::RecordBatch, util::pretty::pretty_format_batches, }; +use arrow_array::new_null_array; use chrono::{Datelike, Duration, TimeDelta}; use datafusion::{ datasource::{physical_plan::ParquetExec, provider_as_source, TableProvider}, @@ -75,6 +76,7 @@ enum Scenario { DecimalLargePrecisionBloomFilter, ByteArray, PeriodsInColumnNames, + WithNullValues, } enum Unit { @@ -630,6 +632,27 @@ fn make_names_batch(name: &str, service_name_values: Vec<&str>) -> RecordBatch { RecordBatch::try_new(schema, vec![Arc::new(name), Arc::new(service_name)]).unwrap() } +/// Return record batch with i8, i16, i32, and i64 sequences with all Null values +fn make_all_null_values() -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("i8", DataType::Int8, true), + Field::new("i16", DataType::Int16, true), + Field::new("i32", DataType::Int32, true), + Field::new("i64", DataType::Int64, true), + ])); + + RecordBatch::try_new( + schema, + vec![ + new_null_array(&DataType::Int8, 5), + new_null_array(&DataType::Int16, 5), + new_null_array(&DataType::Int32, 5), + new_null_array(&DataType::Int64, 5), + ], + ) + .unwrap() +} + fn create_data_batch(scenario: Scenario) -> Vec { match scenario { Scenario::Timestamps => { @@ -799,6 +822,13 @@ fn create_data_batch(scenario: Scenario) -> Vec { ), ] } + Scenario::WithNullValues => { + vec![ + make_all_null_values(), + make_int_batches(1, 6), + make_all_null_values(), + ] + } } } diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 8fc7936552af..29bf1ef0a8d4 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -1262,3 +1262,63 @@ async fn prune_periods_in_column_names() { .test_row_group_prune() .await; } + +#[tokio::test] +async fn test_row_group_with_null_values() { + // Three row groups: + // 1. all Null values + // 2. values from 1 to 5 + // 3. all Null values + + // After pruning, only row group 2 should be selected + RowGroupPruningTest::new() + .with_scenario(Scenario::WithNullValues) + .with_query("SELECT * FROM t WHERE \"i8\" <= 5") + .with_expected_errors(Some(0)) + .with_matched_by_stats(Some(1)) + .with_pruned_by_stats(Some(2)) + .with_expected_rows(5) + .with_matched_by_bloom_filter(Some(0)) + .with_pruned_by_bloom_filter(Some(0)) + .test_row_group_prune() + .await; + + // After pruning, only row group 1,3 should be selected + RowGroupPruningTest::new() + .with_scenario(Scenario::WithNullValues) + .with_query("SELECT * FROM t WHERE \"i8\" is Null") + .with_expected_errors(Some(0)) + .with_matched_by_stats(Some(2)) + .with_pruned_by_stats(Some(1)) + .with_expected_rows(10) + .with_matched_by_bloom_filter(Some(0)) + .with_pruned_by_bloom_filter(Some(0)) + .test_row_group_prune() + .await; + + // After pruning, only row group 2should be selected + RowGroupPruningTest::new() + .with_scenario(Scenario::WithNullValues) + .with_query("SELECT * FROM t WHERE \"i16\" is Not Null") + .with_expected_errors(Some(0)) + .with_matched_by_stats(Some(1)) + .with_pruned_by_stats(Some(2)) + .with_expected_rows(5) + .with_matched_by_bloom_filter(Some(0)) + .with_pruned_by_bloom_filter(Some(0)) + .test_row_group_prune() + .await; + + // All row groups will be pruned + RowGroupPruningTest::new() + .with_scenario(Scenario::WithNullValues) + .with_query("SELECT * FROM t WHERE \"i32\" > 7") + .with_expected_errors(Some(0)) + .with_matched_by_stats(Some(0)) + .with_pruned_by_stats(Some(3)) + .with_expected_rows(0) + .with_matched_by_bloom_filter(Some(0)) + .with_pruned_by_bloom_filter(Some(0)) + .test_row_group_prune() + .await; +} From fdb2d5761c64273ac7326b4a86b052b9bb9c08c7 Mon Sep 17 00:00:00 2001 From: JasonLi Date: Thu, 11 Apr 2024 00:56:09 +0800 Subject: [PATCH 07/39] Improve the performance of ltrim/rtrim/btrim (#10006) * optimize trim function * fix: the second arg is NULL --- datafusion/functions/Cargo.toml | 5 +++ datafusion/functions/benches/ltrim.rs | 50 +++++++++++++++++++++++ datafusion/functions/src/string/btrim.rs | 11 ++++- datafusion/functions/src/string/common.rs | 17 +++++++- datafusion/functions/src/string/ltrim.rs | 11 ++++- datafusion/functions/src/string/rtrim.rs | 11 ++++- 6 files changed, 98 insertions(+), 7 deletions(-) create mode 100644 datafusion/functions/benches/ltrim.rs diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index a6847f3327c0..66f8b3010fd2 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -113,3 +113,8 @@ required-features = ["datetime_expressions"] harness = false name = "substr_index" required-features = ["unicode_expressions"] + +[[bench]] +harness = false +name = "ltrim" +required-features = ["string_expressions"] diff --git a/datafusion/functions/benches/ltrim.rs b/datafusion/functions/benches/ltrim.rs new file mode 100644 index 000000000000..01acb9de3381 --- /dev/null +++ b/datafusion/functions/benches/ltrim.rs @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +extern crate criterion; + +use arrow::array::{ArrayRef, StringArray}; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_common::ScalarValue; +use datafusion_expr::ColumnarValue; +use datafusion_functions::string; +use std::sync::Arc; + +fn create_args(size: usize, characters: &str) -> Vec { + let iter = + std::iter::repeat(format!("{}datafusion{}", characters, characters)).take(size); + let array = Arc::new(StringArray::from_iter_values(iter)) as ArrayRef; + vec![ + ColumnarValue::Array(array), + ColumnarValue::Scalar(ScalarValue::Utf8(Some(characters.to_string()))), + ] +} + +fn criterion_benchmark(c: &mut Criterion) { + let ltrim = string::ltrim(); + for char in ["\"", "Header:"] { + for size in [1024, 4096, 8192] { + let args = create_args(size, char); + c.bench_function(&format!("ltrim {}: {}", char, size), |b| { + b.iter(|| black_box(ltrim.invoke(&args))) + }); + } + } +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions/src/string/btrim.rs b/datafusion/functions/src/string/btrim.rs index b0a85eab6d83..971f7bbd4d92 100644 --- a/datafusion/functions/src/string/btrim.rs +++ b/datafusion/functions/src/string/btrim.rs @@ -24,6 +24,7 @@ use datafusion_common::{exec_err, Result}; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ColumnarValue, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; +use datafusion_physical_expr::functions::Hint; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; @@ -72,8 +73,14 @@ impl ScalarUDFImpl for BTrimFunc { fn invoke(&self, args: &[ColumnarValue]) -> Result { match args[0].data_type() { - DataType::Utf8 => make_scalar_function(btrim::, vec![])(args), - DataType::LargeUtf8 => make_scalar_function(btrim::, vec![])(args), + DataType::Utf8 => make_scalar_function( + btrim::, + vec![Hint::Pad, Hint::AcceptsSingular], + )(args), + DataType::LargeUtf8 => make_scalar_function( + btrim::, + vec![Hint::Pad, Hint::AcceptsSingular], + )(args), other => exec_err!("Unsupported data type {other:?} for function btrim"), } } diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 276aad121df2..2b554db3979f 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -18,7 +18,9 @@ use std::fmt::{Display, Formatter}; use std::sync::Arc; -use arrow::array::{Array, ArrayRef, GenericStringArray, OffsetSizeTrait}; +use arrow::array::{ + new_null_array, Array, ArrayRef, GenericStringArray, OffsetSizeTrait, +}; use arrow::datatypes::DataType; use datafusion_common::cast::as_generic_string_array; @@ -78,6 +80,19 @@ pub(crate) fn general_trim( 2 => { let characters_array = as_generic_string_array::(&args[1])?; + if characters_array.len() == 1 { + if characters_array.is_null(0) { + return Ok(new_null_array(args[0].data_type(), args[0].len())); + } + + let characters = characters_array.value(0); + let result = string_array + .iter() + .map(|item| item.map(|string| func(string, characters))) + .collect::>(); + return Ok(Arc::new(result) as ArrayRef); + } + let result = string_array .iter() .zip(characters_array.iter()) diff --git a/datafusion/functions/src/string/ltrim.rs b/datafusion/functions/src/string/ltrim.rs index ad86259d0d7e..1a6a9d497f66 100644 --- a/datafusion/functions/src/string/ltrim.rs +++ b/datafusion/functions/src/string/ltrim.rs @@ -24,6 +24,7 @@ use datafusion_common::{exec_err, Result}; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ColumnarValue, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; +use datafusion_physical_expr::functions::Hint; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; @@ -70,8 +71,14 @@ impl ScalarUDFImpl for LtrimFunc { fn invoke(&self, args: &[ColumnarValue]) -> Result { match args[0].data_type() { - DataType::Utf8 => make_scalar_function(ltrim::, vec![])(args), - DataType::LargeUtf8 => make_scalar_function(ltrim::, vec![])(args), + DataType::Utf8 => make_scalar_function( + ltrim::, + vec![Hint::Pad, Hint::AcceptsSingular], + )(args), + DataType::LargeUtf8 => make_scalar_function( + ltrim::, + vec![Hint::Pad, Hint::AcceptsSingular], + )(args), other => exec_err!("Unsupported data type {other:?} for function ltrim"), } } diff --git a/datafusion/functions/src/string/rtrim.rs b/datafusion/functions/src/string/rtrim.rs index 607e647b2615..e6e93e38c966 100644 --- a/datafusion/functions/src/string/rtrim.rs +++ b/datafusion/functions/src/string/rtrim.rs @@ -24,6 +24,7 @@ use datafusion_common::{exec_err, Result}; use datafusion_expr::TypeSignature::*; use datafusion_expr::{ColumnarValue, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; +use datafusion_physical_expr::functions::Hint; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; @@ -70,8 +71,14 @@ impl ScalarUDFImpl for RtrimFunc { fn invoke(&self, args: &[ColumnarValue]) -> Result { match args[0].data_type() { - DataType::Utf8 => make_scalar_function(rtrim::, vec![])(args), - DataType::LargeUtf8 => make_scalar_function(rtrim::, vec![])(args), + DataType::Utf8 => make_scalar_function( + rtrim::, + vec![Hint::Pad, Hint::AcceptsSingular], + )(args), + DataType::LargeUtf8 => make_scalar_function( + rtrim::, + vec![Hint::Pad, Hint::AcceptsSingular], + )(args), other => exec_err!("Unsupported data type {other:?} for function rtrim"), } } From a13c37d1d0e3cd0a1383d1685e1efdc015bb4bc8 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Wed, 10 Apr 2024 19:34:53 +0200 Subject: [PATCH 08/39] fix: `RepartitionExec` metrics (#10025) `RepartitionExec` is somewhat special. While most execs operate on "input partition = output partition", `RepartitionExec` drives all of its work using input-bound tasks. The metrics "fetch time" and "repartition time" therefore have to be accounted for the input partition, not for the output partition. The only metric that has an input & output partition label is the "send time". Fixes #10015. --- .../physical-plan/src/repartition/mod.rs | 37 ++++++++++--------- 1 file changed, 20 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 2ed5da7ced20..59c71dbf89b4 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -130,8 +130,7 @@ impl RepartitionExecState { }) .collect(); - // TODO: metric input-output mapping is broken - let r_metrics = RepartitionMetrics::new(i, 0, &metrics); + let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input( input.clone(), @@ -411,32 +410,36 @@ struct RepartitionMetrics { fetch_time: metrics::Time, /// Time in nanos to perform repartitioning repartition_time: metrics::Time, - /// Time in nanos for sending resulting batches to channels - send_time: metrics::Time, + /// Time in nanos for sending resulting batches to channels. + /// + /// One metric per output partition. + send_time: Vec, } impl RepartitionMetrics { pub fn new( - output_partition: usize, input_partition: usize, + num_output_partitions: usize, metrics: &ExecutionPlanMetricsSet, ) -> Self { - let label = metrics::Label::new("inputPartition", input_partition.to_string()); - // Time in nanos to execute child operator and fetch batches - let fetch_time = MetricBuilder::new(metrics) - .with_label(label.clone()) - .subset_time("fetch_time", output_partition); + let fetch_time = + MetricBuilder::new(metrics).subset_time("fetch_time", input_partition); // Time in nanos to perform repartitioning - let repart_time = MetricBuilder::new(metrics) - .with_label(label.clone()) - .subset_time("repart_time", output_partition); + let repart_time = + MetricBuilder::new(metrics).subset_time("repart_time", input_partition); // Time in nanos for sending resulting batches to channels - let send_time = MetricBuilder::new(metrics) - .with_label(label) - .subset_time("send_time", output_partition); + let send_time = (0..num_output_partitions) + .map(|output_partition| { + let label = + metrics::Label::new("outputPartition", output_partition.to_string()); + MetricBuilder::new(metrics) + .with_label(label) + .subset_time("send_time", input_partition) + }) + .collect(); Self { fetch_time, @@ -786,7 +789,7 @@ impl RepartitionExec { let (partition, batch) = res?; let size = batch.get_array_memory_size(); - let timer = metrics.send_time.timer(); + let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it if let Some((tx, reservation)) = output_channels.get_mut(&partition) { reservation.lock().try_grow(size)?; From 69595a48458715aadffc56974665ebbafee35bd7 Mon Sep 17 00:00:00 2001 From: JasonLi Date: Thu, 11 Apr 2024 04:10:27 +0800 Subject: [PATCH 09/39] modify emit() of TopK (#10030) --- datafusion/physical-plan/src/topk/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 9120566273d3..6a77bfaf3ccd 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -208,7 +208,7 @@ impl TopK { // break into record batches as needed let mut batches = vec![]; loop { - if batch.num_rows() < batch_size { + if batch.num_rows() <= batch_size { batches.push(Ok(batch)); break; } else { From b9759b9810a05b7993c0357a5346197395cfd4cc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 10 Apr 2024 17:31:41 -0400 Subject: [PATCH 10/39] Consolidate LogicalPlan tree node walking/rewriting code into one module (#10034) --- datafusion/expr/src/logical_plan/plan.rs | 515 +----------------- datafusion/expr/src/logical_plan/tree_node.rs | 511 ++++++++++++++++- 2 files changed, 515 insertions(+), 511 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 7bad034a11ea..d16dfb140353 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -25,9 +25,7 @@ use std::sync::Arc; use super::dml::CopyTo; use super::DdlStatement; use crate::builder::change_redundant_column; -use crate::expr::{ - Alias, Exists, InSubquery, Placeholder, Sort as SortExpr, WindowFunction, -}; +use crate::expr::{Alias, Placeholder, Sort as SortExpr, WindowFunction}; use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; @@ -44,19 +42,16 @@ use crate::{ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::{ - Transformed, TransformedResult, TreeNode, TreeNodeIterator, TreeNodeRecursion, - TreeNodeRewriter, TreeNodeVisitor, + Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; use datafusion_common::{ - aggregate_functional_dependencies, internal_err, map_until_stop_and_collect, - plan_err, Column, Constraints, DFSchema, DFSchemaRef, DataFusionError, Dependency, - FunctionalDependence, FunctionalDependencies, ParamValues, Result, TableReference, - UnnestOptions, + aggregate_functional_dependencies, internal_err, plan_err, Column, Constraints, + DFSchema, DFSchemaRef, DataFusionError, Dependency, FunctionalDependence, + FunctionalDependencies, ParamValues, Result, TableReference, UnnestOptions, }; // backwards compatibility use crate::display::PgJsonVisitor; -use crate::tree_node::transform_option_vec; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -315,314 +310,6 @@ impl LogicalPlan { err } - /// Calls `f` on all expressions in the current `LogicalPlan` node. - /// - /// Note this does not include expressions in child `LogicalPlan` nodes. - pub fn apply_expressions Result>( - &self, - mut f: F, - ) -> Result { - match self { - LogicalPlan::Projection(Projection { expr, .. }) => { - expr.iter().apply_until_stop(f) - } - LogicalPlan::Values(Values { values, .. }) => values - .iter() - .apply_until_stop(|value| value.iter().apply_until_stop(&mut f)), - LogicalPlan::Filter(Filter { predicate, .. }) => f(predicate), - LogicalPlan::Repartition(Repartition { - partitioning_scheme, - .. - }) => match partitioning_scheme { - Partitioning::Hash(expr, _) | Partitioning::DistributeBy(expr) => { - expr.iter().apply_until_stop(f) - } - Partitioning::RoundRobinBatch(_) => Ok(TreeNodeRecursion::Continue), - }, - LogicalPlan::Window(Window { window_expr, .. }) => { - window_expr.iter().apply_until_stop(f) - } - LogicalPlan::Aggregate(Aggregate { - group_expr, - aggr_expr, - .. - }) => group_expr - .iter() - .chain(aggr_expr.iter()) - .apply_until_stop(f), - // There are two part of expression for join, equijoin(on) and non-equijoin(filter). - // 1. the first part is `on.len()` equijoin expressions, and the struct of each expr is `left-on = right-on`. - // 2. the second part is non-equijoin(filter). - LogicalPlan::Join(Join { on, filter, .. }) => { - on.iter() - // TODO: why we need to create an `Expr::eq`? Cloning `Expr` is costly... - // it not ideal to create an expr here to analyze them, but could cache it on the Join itself - .map(|(l, r)| Expr::eq(l.clone(), r.clone())) - .apply_until_stop(|e| f(&e))? - .visit_sibling(|| filter.iter().apply_until_stop(f)) - } - LogicalPlan::Sort(Sort { expr, .. }) => expr.iter().apply_until_stop(f), - LogicalPlan::Extension(extension) => { - // would be nice to avoid this copy -- maybe can - // update extension to just observer Exprs - extension.node.expressions().iter().apply_until_stop(f) - } - LogicalPlan::TableScan(TableScan { filters, .. }) => { - filters.iter().apply_until_stop(f) - } - LogicalPlan::Unnest(Unnest { column, .. }) => { - f(&Expr::Column(column.clone())) - } - LogicalPlan::Distinct(Distinct::On(DistinctOn { - on_expr, - select_expr, - sort_expr, - .. - })) => on_expr - .iter() - .chain(select_expr.iter()) - .chain(sort_expr.iter().flatten()) - .apply_until_stop(f), - // plans without expressions - LogicalPlan::EmptyRelation(_) - | LogicalPlan::RecursiveQuery(_) - | LogicalPlan::Subquery(_) - | LogicalPlan::SubqueryAlias(_) - | LogicalPlan::Limit(_) - | LogicalPlan::Statement(_) - | LogicalPlan::CrossJoin(_) - | LogicalPlan::Analyze(_) - | LogicalPlan::Explain(_) - | LogicalPlan::Union(_) - | LogicalPlan::Distinct(Distinct::All(_)) - | LogicalPlan::Dml(_) - | LogicalPlan::Ddl(_) - | LogicalPlan::Copy(_) - | LogicalPlan::DescribeTable(_) - | LogicalPlan::Prepare(_) => Ok(TreeNodeRecursion::Continue), - } - } - - /// Rewrites all expressions in the current `LogicalPlan` node using `f`. - /// - /// Returns the current node. - /// - /// Note this does not include expressions in child `LogicalPlan` nodes. - pub fn map_expressions Result>>( - self, - mut f: F, - ) -> Result> { - Ok(match self { - LogicalPlan::Projection(Projection { - expr, - input, - schema, - }) => expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|expr| { - LogicalPlan::Projection(Projection { - expr, - input, - schema, - }) - }), - LogicalPlan::Values(Values { schema, values }) => values - .into_iter() - .map_until_stop_and_collect(|value| { - value.into_iter().map_until_stop_and_collect(&mut f) - })? - .update_data(|values| LogicalPlan::Values(Values { schema, values })), - LogicalPlan::Filter(Filter { predicate, input }) => f(predicate)? - .update_data(|predicate| { - LogicalPlan::Filter(Filter { predicate, input }) - }), - LogicalPlan::Repartition(Repartition { - input, - partitioning_scheme, - }) => match partitioning_scheme { - Partitioning::Hash(expr, usize) => expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|expr| Partitioning::Hash(expr, usize)), - Partitioning::DistributeBy(expr) => expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(Partitioning::DistributeBy), - Partitioning::RoundRobinBatch(_) => Transformed::no(partitioning_scheme), - } - .update_data(|partitioning_scheme| { - LogicalPlan::Repartition(Repartition { - input, - partitioning_scheme, - }) - }), - LogicalPlan::Window(Window { - input, - window_expr, - schema, - }) => window_expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|window_expr| { - LogicalPlan::Window(Window { - input, - window_expr, - schema, - }) - }), - LogicalPlan::Aggregate(Aggregate { - input, - group_expr, - aggr_expr, - schema, - }) => map_until_stop_and_collect!( - group_expr.into_iter().map_until_stop_and_collect(&mut f), - aggr_expr, - aggr_expr.into_iter().map_until_stop_and_collect(&mut f) - )? - .update_data(|(group_expr, aggr_expr)| { - LogicalPlan::Aggregate(Aggregate { - input, - group_expr, - aggr_expr, - schema, - }) - }), - - // There are two part of expression for join, equijoin(on) and non-equijoin(filter). - // 1. the first part is `on.len()` equijoin expressions, and the struct of each expr is `left-on = right-on`. - // 2. the second part is non-equijoin(filter). - LogicalPlan::Join(Join { - left, - right, - on, - filter, - join_type, - join_constraint, - schema, - null_equals_null, - }) => map_until_stop_and_collect!( - on.into_iter().map_until_stop_and_collect( - |on| map_until_stop_and_collect!(f(on.0), on.1, f(on.1)) - ), - filter, - filter.map_or(Ok::<_, DataFusionError>(Transformed::no(None)), |e| { - Ok(f(e)?.update_data(Some)) - }) - )? - .update_data(|(on, filter)| { - LogicalPlan::Join(Join { - left, - right, - on, - filter, - join_type, - join_constraint, - schema, - null_equals_null, - }) - }), - LogicalPlan::Sort(Sort { expr, input, fetch }) => expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|expr| LogicalPlan::Sort(Sort { expr, input, fetch })), - LogicalPlan::Extension(Extension { node }) => { - // would be nice to avoid this copy -- maybe can - // update extension to just observer Exprs - node.expressions() - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|exprs| { - LogicalPlan::Extension(Extension { - node: UserDefinedLogicalNode::from_template( - node.as_ref(), - exprs.as_slice(), - node.inputs() - .into_iter() - .cloned() - .collect::>() - .as_slice(), - ), - }) - }) - } - LogicalPlan::TableScan(TableScan { - table_name, - source, - projection, - projected_schema, - filters, - fetch, - }) => filters - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|filters| { - LogicalPlan::TableScan(TableScan { - table_name, - source, - projection, - projected_schema, - filters, - fetch, - }) - }), - LogicalPlan::Unnest(Unnest { - input, - column, - schema, - options, - }) => f(Expr::Column(column))?.map_data(|column| match column { - Expr::Column(column) => Ok(LogicalPlan::Unnest(Unnest { - input, - column, - schema, - options, - })), - _ => internal_err!("Transformation should return Column"), - })?, - LogicalPlan::Distinct(Distinct::On(DistinctOn { - on_expr, - select_expr, - sort_expr, - input, - schema, - })) => map_until_stop_and_collect!( - on_expr.into_iter().map_until_stop_and_collect(&mut f), - select_expr, - select_expr.into_iter().map_until_stop_and_collect(&mut f), - sort_expr, - transform_option_vec(sort_expr, &mut f) - )? - .update_data(|(on_expr, select_expr, sort_expr)| { - LogicalPlan::Distinct(Distinct::On(DistinctOn { - on_expr, - select_expr, - sort_expr, - input, - schema, - })) - }), - // plans without expressions - LogicalPlan::EmptyRelation(_) - | LogicalPlan::RecursiveQuery(_) - | LogicalPlan::Subquery(_) - | LogicalPlan::SubqueryAlias(_) - | LogicalPlan::Limit(_) - | LogicalPlan::Statement(_) - | LogicalPlan::CrossJoin(_) - | LogicalPlan::Analyze(_) - | LogicalPlan::Explain(_) - | LogicalPlan::Union(_) - | LogicalPlan::Distinct(Distinct::All(_)) - | LogicalPlan::Dml(_) - | LogicalPlan::Ddl(_) - | LogicalPlan::Copy(_) - | LogicalPlan::DescribeTable(_) - | LogicalPlan::Prepare(_) => Transformed::no(self), - }) - } - /// Returns all inputs / children of this `LogicalPlan` node. /// /// Note does not include inputs to inputs, or subqueries. @@ -1354,192 +1041,7 @@ impl LogicalPlan { } } -/// This macro is used to determine continuation during combined transforming -/// traversals. -macro_rules! handle_transform_recursion { - ($F_DOWN:expr, $F_CHILD:expr, $F_UP:expr) => {{ - $F_DOWN? - .transform_children(|n| n.map_subqueries($F_CHILD))? - .transform_sibling(|n| n.map_children($F_CHILD))? - .transform_parent($F_UP) - }}; -} - -macro_rules! handle_transform_recursion_down { - ($F_DOWN:expr, $F_CHILD:expr) => {{ - $F_DOWN? - .transform_children(|n| n.map_subqueries($F_CHILD))? - .transform_sibling(|n| n.map_children($F_CHILD)) - }}; -} - -macro_rules! handle_transform_recursion_up { - ($SELF:expr, $F_CHILD:expr, $F_UP:expr) => {{ - $SELF - .map_subqueries($F_CHILD)? - .transform_sibling(|n| n.map_children($F_CHILD))? - .transform_parent(|n| $F_UP(n)) - }}; -} - impl LogicalPlan { - /// Visits a plan similarly to [`Self::visit`], but including embedded subqueries. - pub fn visit_with_subqueries>( - &self, - visitor: &mut V, - ) -> Result { - visitor - .f_down(self)? - .visit_children(|| { - self.apply_subqueries(|c| c.visit_with_subqueries(visitor)) - })? - .visit_sibling(|| self.apply_children(|c| c.visit_with_subqueries(visitor)))? - .visit_parent(|| visitor.f_up(self)) - } - - /// Rewrites a plan similarly t [`Self::visit`], but including embedded subqueries. - pub fn rewrite_with_subqueries>( - self, - rewriter: &mut R, - ) -> Result> { - handle_transform_recursion!( - rewriter.f_down(self), - |c| c.rewrite_with_subqueries(rewriter), - |n| rewriter.f_up(n) - ) - } - - /// Calls `f` recursively on all children of the `LogicalPlan` node. - /// - /// Unlike [`Self::apply`], this method *does* includes `LogicalPlan`s that - /// are referenced in `Expr`s - pub fn apply_with_subqueries Result>( - &self, - f: &mut F, - ) -> Result { - f(self)? - .visit_children(|| self.apply_subqueries(|c| c.apply_with_subqueries(f)))? - .visit_sibling(|| self.apply_children(|c| c.apply_with_subqueries(f))) - } - - pub fn transform_with_subqueries Result>>( - self, - f: &F, - ) -> Result> { - self.transform_up_with_subqueries(f) - } - - pub fn transform_down_with_subqueries Result>>( - self, - f: &F, - ) -> Result> { - handle_transform_recursion_down!(f(self), |c| c.transform_down_with_subqueries(f)) - } - - pub fn transform_down_mut_with_subqueries< - F: FnMut(Self) -> Result>, - >( - self, - f: &mut F, - ) -> Result> { - handle_transform_recursion_down!(f(self), |c| c - .transform_down_mut_with_subqueries(f)) - } - - pub fn transform_up_with_subqueries Result>>( - self, - f: &F, - ) -> Result> { - handle_transform_recursion_up!(self, |c| c.transform_up_with_subqueries(f), f) - } - - pub fn transform_up_mut_with_subqueries< - F: FnMut(Self) -> Result>, - >( - self, - f: &mut F, - ) -> Result> { - handle_transform_recursion_up!(self, |c| c.transform_up_mut_with_subqueries(f), f) - } - - pub fn transform_down_up_with_subqueries< - FD: FnMut(Self) -> Result>, - FU: FnMut(Self) -> Result>, - >( - self, - f_down: &mut FD, - f_up: &mut FU, - ) -> Result> { - handle_transform_recursion!( - f_down(self), - |c| c.transform_down_up_with_subqueries(f_down, f_up), - f_up - ) - } - - /// Calls `f` on all subqueries referenced in expressions of the current - /// `LogicalPlan` node. - pub fn apply_subqueries Result>( - &self, - mut f: F, - ) -> Result { - self.apply_expressions(|expr| { - expr.apply(&mut |expr| match expr { - Expr::Exists(Exists { subquery, .. }) - | Expr::InSubquery(InSubquery { subquery, .. }) - | Expr::ScalarSubquery(subquery) => { - // use a synthetic plan so the collector sees a - // LogicalPlan::Subquery (even though it is - // actually a Subquery alias) - f(&LogicalPlan::Subquery(subquery.clone())) - } - _ => Ok(TreeNodeRecursion::Continue), - }) - }) - } - - /// Rewrites all subquery `LogicalPlan` in the current `LogicalPlan` node - /// using `f`. - /// - /// Returns the current node. - pub fn map_subqueries Result>>( - self, - mut f: F, - ) -> Result> { - self.map_expressions(|expr| { - expr.transform_down_mut(&mut |expr| match expr { - Expr::Exists(Exists { subquery, negated }) => { - f(LogicalPlan::Subquery(subquery))?.map_data(|s| match s { - LogicalPlan::Subquery(subquery) => { - Ok(Expr::Exists(Exists { subquery, negated })) - } - _ => internal_err!("Transformation should return Subquery"), - }) - } - Expr::InSubquery(InSubquery { - expr, - subquery, - negated, - }) => f(LogicalPlan::Subquery(subquery))?.map_data(|s| match s { - LogicalPlan::Subquery(subquery) => Ok(Expr::InSubquery(InSubquery { - expr, - subquery, - negated, - })), - _ => internal_err!("Transformation should return Subquery"), - }), - Expr::ScalarSubquery(subquery) => f(LogicalPlan::Subquery(subquery))? - .map_data(|s| match s { - LogicalPlan::Subquery(subquery) => { - Ok(Expr::ScalarSubquery(subquery)) - } - _ => internal_err!("Transformation should return Subquery"), - }), - _ => Ok(Transformed::no(expr)), - }) - }) - } - /// Return a `LogicalPlan` with all placeholders (e.g $1 $2, /// ...) replaced with corresponding values provided in /// `params_values` @@ -1623,10 +1125,11 @@ impl LogicalPlan { }) .data() } -} -// Various implementations for printing out LogicalPlans -impl LogicalPlan { + // ------------ + // Various implementations for printing out LogicalPlans + // ------------ + /// Return a `format`able structure that produces a single line /// per node. /// diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 415343f88685..1eb9d50277dd 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -38,16 +38,22 @@ //! * [`LogicalPlan::expressions`]: Return a copy of the plan's expressions use crate::{ dml::CopyTo, Aggregate, Analyze, CreateMemoryTable, CreateView, CrossJoin, - DdlStatement, Distinct, DistinctOn, DmlStatement, Explain, Extension, Filter, Join, - Limit, LogicalPlan, Prepare, Projection, RecursiveQuery, Repartition, Sort, Subquery, - SubqueryAlias, Union, Unnest, Window, + DdlStatement, Distinct, DistinctOn, DmlStatement, Explain, Expr, Extension, Filter, + Join, Limit, LogicalPlan, Partitioning, Prepare, Projection, RecursiveQuery, + Repartition, Sort, Subquery, SubqueryAlias, TableScan, Union, Unnest, + UserDefinedLogicalNode, Values, Window, }; use std::sync::Arc; +use crate::expr::{Exists, InSubquery}; +use crate::tree_node::transform_option_vec; use datafusion_common::tree_node::{ - Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, + Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, TreeNodeRewriter, + TreeNodeVisitor, +}; +use datafusion_common::{ + internal_err, map_until_stop_and_collect, DataFusionError, Result, }; -use datafusion_common::{map_until_stop_and_collect, Result}; impl TreeNode for LogicalPlan { fn apply_children Result>( @@ -413,3 +419,498 @@ where }) }) } + +/// This macro is used to determine continuation during combined transforming +/// traversals. +macro_rules! handle_transform_recursion { + ($F_DOWN:expr, $F_CHILD:expr, $F_UP:expr) => {{ + $F_DOWN? + .transform_children(|n| n.map_subqueries($F_CHILD))? + .transform_sibling(|n| n.map_children($F_CHILD))? + .transform_parent($F_UP) + }}; +} + +macro_rules! handle_transform_recursion_down { + ($F_DOWN:expr, $F_CHILD:expr) => {{ + $F_DOWN? + .transform_children(|n| n.map_subqueries($F_CHILD))? + .transform_sibling(|n| n.map_children($F_CHILD)) + }}; +} + +macro_rules! handle_transform_recursion_up { + ($SELF:expr, $F_CHILD:expr, $F_UP:expr) => {{ + $SELF + .map_subqueries($F_CHILD)? + .transform_sibling(|n| n.map_children($F_CHILD))? + .transform_parent(|n| $F_UP(n)) + }}; +} + +impl LogicalPlan { + /// Calls `f` on all expressions in the current `LogicalPlan` node. + /// + /// Note this does not include expressions in child `LogicalPlan` nodes. + pub fn apply_expressions Result>( + &self, + mut f: F, + ) -> Result { + match self { + LogicalPlan::Projection(Projection { expr, .. }) => { + expr.iter().apply_until_stop(f) + } + LogicalPlan::Values(Values { values, .. }) => values + .iter() + .apply_until_stop(|value| value.iter().apply_until_stop(&mut f)), + LogicalPlan::Filter(Filter { predicate, .. }) => f(predicate), + LogicalPlan::Repartition(Repartition { + partitioning_scheme, + .. + }) => match partitioning_scheme { + Partitioning::Hash(expr, _) | Partitioning::DistributeBy(expr) => { + expr.iter().apply_until_stop(f) + } + Partitioning::RoundRobinBatch(_) => Ok(TreeNodeRecursion::Continue), + }, + LogicalPlan::Window(Window { window_expr, .. }) => { + window_expr.iter().apply_until_stop(f) + } + LogicalPlan::Aggregate(Aggregate { + group_expr, + aggr_expr, + .. + }) => group_expr + .iter() + .chain(aggr_expr.iter()) + .apply_until_stop(f), + // There are two part of expression for join, equijoin(on) and non-equijoin(filter). + // 1. the first part is `on.len()` equijoin expressions, and the struct of each expr is `left-on = right-on`. + // 2. the second part is non-equijoin(filter). + LogicalPlan::Join(Join { on, filter, .. }) => { + on.iter() + // TODO: why we need to create an `Expr::eq`? Cloning `Expr` is costly... + // it not ideal to create an expr here to analyze them, but could cache it on the Join itself + .map(|(l, r)| Expr::eq(l.clone(), r.clone())) + .apply_until_stop(|e| f(&e))? + .visit_sibling(|| filter.iter().apply_until_stop(f)) + } + LogicalPlan::Sort(Sort { expr, .. }) => expr.iter().apply_until_stop(f), + LogicalPlan::Extension(extension) => { + // would be nice to avoid this copy -- maybe can + // update extension to just observer Exprs + extension.node.expressions().iter().apply_until_stop(f) + } + LogicalPlan::TableScan(TableScan { filters, .. }) => { + filters.iter().apply_until_stop(f) + } + LogicalPlan::Unnest(Unnest { column, .. }) => { + f(&Expr::Column(column.clone())) + } + LogicalPlan::Distinct(Distinct::On(DistinctOn { + on_expr, + select_expr, + sort_expr, + .. + })) => on_expr + .iter() + .chain(select_expr.iter()) + .chain(sort_expr.iter().flatten()) + .apply_until_stop(f), + // plans without expressions + LogicalPlan::EmptyRelation(_) + | LogicalPlan::RecursiveQuery(_) + | LogicalPlan::Subquery(_) + | LogicalPlan::SubqueryAlias(_) + | LogicalPlan::Limit(_) + | LogicalPlan::Statement(_) + | LogicalPlan::CrossJoin(_) + | LogicalPlan::Analyze(_) + | LogicalPlan::Explain(_) + | LogicalPlan::Union(_) + | LogicalPlan::Distinct(Distinct::All(_)) + | LogicalPlan::Dml(_) + | LogicalPlan::Ddl(_) + | LogicalPlan::Copy(_) + | LogicalPlan::DescribeTable(_) + | LogicalPlan::Prepare(_) => Ok(TreeNodeRecursion::Continue), + } + } + + /// Rewrites all expressions in the current `LogicalPlan` node using `f`. + /// + /// Returns the current node. + /// + /// Note this does not include expressions in child `LogicalPlan` nodes. + pub fn map_expressions Result>>( + self, + mut f: F, + ) -> Result> { + Ok(match self { + LogicalPlan::Projection(Projection { + expr, + input, + schema, + }) => expr + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(|expr| { + LogicalPlan::Projection(Projection { + expr, + input, + schema, + }) + }), + LogicalPlan::Values(Values { schema, values }) => values + .into_iter() + .map_until_stop_and_collect(|value| { + value.into_iter().map_until_stop_and_collect(&mut f) + })? + .update_data(|values| LogicalPlan::Values(Values { schema, values })), + LogicalPlan::Filter(Filter { predicate, input }) => f(predicate)? + .update_data(|predicate| { + LogicalPlan::Filter(Filter { predicate, input }) + }), + LogicalPlan::Repartition(Repartition { + input, + partitioning_scheme, + }) => match partitioning_scheme { + Partitioning::Hash(expr, usize) => expr + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(|expr| Partitioning::Hash(expr, usize)), + Partitioning::DistributeBy(expr) => expr + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(Partitioning::DistributeBy), + Partitioning::RoundRobinBatch(_) => Transformed::no(partitioning_scheme), + } + .update_data(|partitioning_scheme| { + LogicalPlan::Repartition(Repartition { + input, + partitioning_scheme, + }) + }), + LogicalPlan::Window(Window { + input, + window_expr, + schema, + }) => window_expr + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(|window_expr| { + LogicalPlan::Window(Window { + input, + window_expr, + schema, + }) + }), + LogicalPlan::Aggregate(Aggregate { + input, + group_expr, + aggr_expr, + schema, + }) => map_until_stop_and_collect!( + group_expr.into_iter().map_until_stop_and_collect(&mut f), + aggr_expr, + aggr_expr.into_iter().map_until_stop_and_collect(&mut f) + )? + .update_data(|(group_expr, aggr_expr)| { + LogicalPlan::Aggregate(Aggregate { + input, + group_expr, + aggr_expr, + schema, + }) + }), + + // There are two part of expression for join, equijoin(on) and non-equijoin(filter). + // 1. the first part is `on.len()` equijoin expressions, and the struct of each expr is `left-on = right-on`. + // 2. the second part is non-equijoin(filter). + LogicalPlan::Join(Join { + left, + right, + on, + filter, + join_type, + join_constraint, + schema, + null_equals_null, + }) => map_until_stop_and_collect!( + on.into_iter().map_until_stop_and_collect( + |on| map_until_stop_and_collect!(f(on.0), on.1, f(on.1)) + ), + filter, + filter.map_or(Ok::<_, DataFusionError>(Transformed::no(None)), |e| { + Ok(f(e)?.update_data(Some)) + }) + )? + .update_data(|(on, filter)| { + LogicalPlan::Join(Join { + left, + right, + on, + filter, + join_type, + join_constraint, + schema, + null_equals_null, + }) + }), + LogicalPlan::Sort(Sort { expr, input, fetch }) => expr + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(|expr| LogicalPlan::Sort(Sort { expr, input, fetch })), + LogicalPlan::Extension(Extension { node }) => { + // would be nice to avoid this copy -- maybe can + // update extension to just observer Exprs + node.expressions() + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(|exprs| { + LogicalPlan::Extension(Extension { + node: UserDefinedLogicalNode::from_template( + node.as_ref(), + exprs.as_slice(), + node.inputs() + .into_iter() + .cloned() + .collect::>() + .as_slice(), + ), + }) + }) + } + LogicalPlan::TableScan(TableScan { + table_name, + source, + projection, + projected_schema, + filters, + fetch, + }) => filters + .into_iter() + .map_until_stop_and_collect(f)? + .update_data(|filters| { + LogicalPlan::TableScan(TableScan { + table_name, + source, + projection, + projected_schema, + filters, + fetch, + }) + }), + LogicalPlan::Unnest(Unnest { + input, + column, + schema, + options, + }) => f(Expr::Column(column))?.map_data(|column| match column { + Expr::Column(column) => Ok(LogicalPlan::Unnest(Unnest { + input, + column, + schema, + options, + })), + _ => internal_err!("Transformation should return Column"), + })?, + LogicalPlan::Distinct(Distinct::On(DistinctOn { + on_expr, + select_expr, + sort_expr, + input, + schema, + })) => map_until_stop_and_collect!( + on_expr.into_iter().map_until_stop_and_collect(&mut f), + select_expr, + select_expr.into_iter().map_until_stop_and_collect(&mut f), + sort_expr, + transform_option_vec(sort_expr, &mut f) + )? + .update_data(|(on_expr, select_expr, sort_expr)| { + LogicalPlan::Distinct(Distinct::On(DistinctOn { + on_expr, + select_expr, + sort_expr, + input, + schema, + })) + }), + // plans without expressions + LogicalPlan::EmptyRelation(_) + | LogicalPlan::RecursiveQuery(_) + | LogicalPlan::Subquery(_) + | LogicalPlan::SubqueryAlias(_) + | LogicalPlan::Limit(_) + | LogicalPlan::Statement(_) + | LogicalPlan::CrossJoin(_) + | LogicalPlan::Analyze(_) + | LogicalPlan::Explain(_) + | LogicalPlan::Union(_) + | LogicalPlan::Distinct(Distinct::All(_)) + | LogicalPlan::Dml(_) + | LogicalPlan::Ddl(_) + | LogicalPlan::Copy(_) + | LogicalPlan::DescribeTable(_) + | LogicalPlan::Prepare(_) => Transformed::no(self), + }) + } + + /// Visits a plan similarly to [`Self::visit`], but including embedded subqueries. + pub fn visit_with_subqueries>( + &self, + visitor: &mut V, + ) -> Result { + visitor + .f_down(self)? + .visit_children(|| { + self.apply_subqueries(|c| c.visit_with_subqueries(visitor)) + })? + .visit_sibling(|| self.apply_children(|c| c.visit_with_subqueries(visitor)))? + .visit_parent(|| visitor.f_up(self)) + } + + /// Rewrites a plan similarly t [`Self::visit`], but including embedded subqueries. + pub fn rewrite_with_subqueries>( + self, + rewriter: &mut R, + ) -> Result> { + handle_transform_recursion!( + rewriter.f_down(self), + |c| c.rewrite_with_subqueries(rewriter), + |n| rewriter.f_up(n) + ) + } + + /// Calls `f` recursively on all children of the `LogicalPlan` node. + /// + /// Unlike [`Self::apply`], this method *does* includes `LogicalPlan`s that + /// are referenced in `Expr`s + pub fn apply_with_subqueries Result>( + &self, + f: &mut F, + ) -> Result { + f(self)? + .visit_children(|| self.apply_subqueries(|c| c.apply_with_subqueries(f)))? + .visit_sibling(|| self.apply_children(|c| c.apply_with_subqueries(f))) + } + + pub fn transform_with_subqueries Result>>( + self, + f: &F, + ) -> Result> { + self.transform_up_with_subqueries(f) + } + + pub fn transform_down_with_subqueries Result>>( + self, + f: &F, + ) -> Result> { + handle_transform_recursion_down!(f(self), |c| c.transform_down_with_subqueries(f)) + } + + pub fn transform_down_mut_with_subqueries< + F: FnMut(Self) -> Result>, + >( + self, + f: &mut F, + ) -> Result> { + handle_transform_recursion_down!(f(self), |c| c + .transform_down_mut_with_subqueries(f)) + } + + pub fn transform_up_with_subqueries Result>>( + self, + f: &F, + ) -> Result> { + handle_transform_recursion_up!(self, |c| c.transform_up_with_subqueries(f), f) + } + + pub fn transform_up_mut_with_subqueries< + F: FnMut(Self) -> Result>, + >( + self, + f: &mut F, + ) -> Result> { + handle_transform_recursion_up!(self, |c| c.transform_up_mut_with_subqueries(f), f) + } + + pub fn transform_down_up_with_subqueries< + FD: FnMut(Self) -> Result>, + FU: FnMut(Self) -> Result>, + >( + self, + f_down: &mut FD, + f_up: &mut FU, + ) -> Result> { + handle_transform_recursion!( + f_down(self), + |c| c.transform_down_up_with_subqueries(f_down, f_up), + f_up + ) + } + + /// Calls `f` on all subqueries referenced in expressions of the current + /// `LogicalPlan` node. + pub fn apply_subqueries Result>( + &self, + mut f: F, + ) -> Result { + self.apply_expressions(|expr| { + expr.apply(&mut |expr| match expr { + Expr::Exists(Exists { subquery, .. }) + | Expr::InSubquery(InSubquery { subquery, .. }) + | Expr::ScalarSubquery(subquery) => { + // use a synthetic plan so the collector sees a + // LogicalPlan::Subquery (even though it is + // actually a Subquery alias) + f(&LogicalPlan::Subquery(subquery.clone())) + } + _ => Ok(TreeNodeRecursion::Continue), + }) + }) + } + + /// Rewrites all subquery `LogicalPlan` in the current `LogicalPlan` node + /// using `f`. + /// + /// Returns the current node. + pub fn map_subqueries Result>>( + self, + mut f: F, + ) -> Result> { + self.map_expressions(|expr| { + expr.transform_down_mut(&mut |expr| match expr { + Expr::Exists(Exists { subquery, negated }) => { + f(LogicalPlan::Subquery(subquery))?.map_data(|s| match s { + LogicalPlan::Subquery(subquery) => { + Ok(Expr::Exists(Exists { subquery, negated })) + } + _ => internal_err!("Transformation should return Subquery"), + }) + } + Expr::InSubquery(InSubquery { + expr, + subquery, + negated, + }) => f(LogicalPlan::Subquery(subquery))?.map_data(|s| match s { + LogicalPlan::Subquery(subquery) => Ok(Expr::InSubquery(InSubquery { + expr, + subquery, + negated, + })), + _ => internal_err!("Transformation should return Subquery"), + }), + Expr::ScalarSubquery(subquery) => f(LogicalPlan::Subquery(subquery))? + .map_data(|s| match s { + LogicalPlan::Subquery(subquery) => { + Ok(Expr::ScalarSubquery(subquery)) + } + _ => internal_err!("Transformation should return Subquery"), + }), + _ => Ok(Transformed::no(expr)), + }) + }) + } +} From 58e0b599a2b7def75ce7da29632f6a859abef551 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 11 Apr 2024 03:27:07 -0400 Subject: [PATCH 11/39] Introduce `OptimizerRule::rewrite` to rewrite in place, Rewrite `SimplifyExprs` to avoid copies (#9954) --- datafusion/core/tests/simplification.rs | 25 +-- datafusion/optimizer/src/optimizer.rs | 50 ++++-- .../simplify_expressions/simplify_exprs.rs | 152 ++++++++++-------- 3 files changed, 139 insertions(+), 88 deletions(-) diff --git a/datafusion/core/tests/simplification.rs b/datafusion/core/tests/simplification.rs index a0bcdda84d64..dc075e669564 100644 --- a/datafusion/core/tests/simplification.rs +++ b/datafusion/core/tests/simplification.rs @@ -32,6 +32,7 @@ use datafusion_expr::{ LogicalPlanBuilder, ScalarUDF, Volatility, }; use datafusion_functions::math; +use datafusion_optimizer::optimizer::Optimizer; use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyExpressions}; use datafusion_optimizer::{OptimizerContext, OptimizerRule}; use std::sync::Arc; @@ -109,14 +110,14 @@ fn test_table_scan() -> LogicalPlan { .expect("building plan") } -fn get_optimized_plan_formatted(plan: &LogicalPlan, date_time: &DateTime) -> String { +fn get_optimized_plan_formatted(plan: LogicalPlan, date_time: &DateTime) -> String { let config = OptimizerContext::new().with_query_execution_start_time(*date_time); - let rule = SimplifyExpressions::new(); - let optimized_plan = rule - .try_optimize(plan, &config) - .unwrap() - .expect("failed to optimize plan"); + // Use Optimizer to do plan traversal + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + let optimizer = Optimizer::with_rules(vec![Arc::new(SimplifyExpressions::new())]); + let optimized_plan = optimizer.optimize(plan, &config, observe).unwrap(); + format!("{optimized_plan:?}") } @@ -238,7 +239,7 @@ fn to_timestamp_expr_folded() -> Result<()> { let expected = "Projection: TimestampNanosecond(1599566400000000000, None) AS to_timestamp(Utf8(\"2020-09-08T12:00:00+00:00\"))\ \n TableScan: test" .to_string(); - let actual = get_optimized_plan_formatted(&plan, &Utc::now()); + let actual = get_optimized_plan_formatted(plan, &Utc::now()); assert_eq!(expected, actual); Ok(()) } @@ -262,7 +263,7 @@ fn now_less_than_timestamp() -> Result<()> { // expression down to a single constant (true) let expected = "Filter: Boolean(true)\ \n TableScan: test"; - let actual = get_optimized_plan_formatted(&plan, &time); + let actual = get_optimized_plan_formatted(plan, &time); assert_eq!(expected, actual); Ok(()) @@ -290,7 +291,7 @@ fn select_date_plus_interval() -> Result<()> { // expression down to a single constant (true) let expected = r#"Projection: Date32("18636") AS to_timestamp(Utf8("2020-09-08T12:05:00+00:00")) + IntervalDayTime("528280977408") TableScan: test"#; - let actual = get_optimized_plan_formatted(&plan, &time); + let actual = get_optimized_plan_formatted(plan, &time); assert_eq!(expected, actual); Ok(()) @@ -308,7 +309,7 @@ fn simplify_project_scalar_fn() -> Result<()> { // after simplify: t.f as "power(t.f, 1.0)" let expected = "Projection: test.f AS power(test.f,Float64(1))\ \n TableScan: test"; - let actual = get_optimized_plan_formatted(&plan, &Utc::now()); + let actual = get_optimized_plan_formatted(plan, &Utc::now()); assert_eq!(expected, actual); Ok(()) } @@ -330,7 +331,7 @@ fn simplify_scan_predicate() -> Result<()> { // before simplify: t.g = power(t.f, 1.0) // after simplify: (t.g = t.f) as "t.g = power(t.f, 1.0)" let expected = "TableScan: test, full_filters=[g = f AS g = power(f,Float64(1))]"; - let actual = get_optimized_plan_formatted(&plan, &Utc::now()); + let actual = get_optimized_plan_formatted(plan, &Utc::now()); assert_eq!(expected, actual); Ok(()) } @@ -461,7 +462,7 @@ fn multiple_now() -> Result<()> { .build()?; // expect the same timestamp appears in both exprs - let actual = get_optimized_plan_formatted(&plan, &time); + let actual = get_optimized_plan_formatted(plan, &time); let expected = format!( "Projection: TimestampNanosecond({}, Some(\"+00:00\")) AS now(), TimestampNanosecond({}, Some(\"+00:00\")) AS t2\ \n TableScan: test", diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 032f9c57321c..ff692681ccd6 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -27,7 +27,7 @@ use datafusion_common::alias::AliasGenerator; use datafusion_common::config::ConfigOptions; use datafusion_common::instant::Instant; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; -use datafusion_common::{DFSchema, DataFusionError, Result}; +use datafusion_common::{internal_err, DFSchema, DataFusionError, Result}; use datafusion_expr::logical_plan::LogicalPlan; use crate::common_subexpr_eliminate::CommonSubexprEliminate; @@ -69,8 +69,12 @@ use crate::utils::log_plan; /// [`SessionState::add_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionState.html#method.add_optimizer_rule pub trait OptimizerRule { - /// Try and rewrite `plan` to an optimized form, returning None if the plan cannot be - /// optimized by this rule. + /// Try and rewrite `plan` to an optimized form, returning None if the plan + /// cannot be optimized by this rule. + /// + /// Note this API will be deprecated in the future as it requires `clone`ing + /// the input plan, which can be expensive. OptimizerRules should implement + /// [`Self::rewrite`] instead. fn try_optimize( &self, plan: &LogicalPlan, @@ -80,12 +84,31 @@ pub trait OptimizerRule { /// A human readable name for this optimizer rule fn name(&self) -> &str; - /// How should the rule be applied by the optimizer? See comments on [`ApplyOrder`] for details. + /// How should the rule be applied by the optimizer? See comments on + /// [`ApplyOrder`] for details. /// - /// If a rule use default None, it should traverse recursively plan inside itself + /// If returns `None`, the default, the rule must handle recursion itself fn apply_order(&self) -> Option { None } + + /// Does this rule support rewriting owned plans (rather than by reference)? + fn supports_rewrite(&self) -> bool { + false + } + + /// Try to rewrite `plan` to an optimized form, returning `Transformed::yes` + /// if the plan was rewritten and `Transformed::no` if it was not. + /// + /// Note: this function is only called if [`Self::supports_rewrite`] returns + /// true. Otherwise the Optimizer calls [`Self::try_optimize`] + fn rewrite( + &self, + _plan: LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result, DataFusionError> { + internal_err!("rewrite is not implemented for {}", self.name()) + } } /// Options to control the DataFusion Optimizer. @@ -298,12 +321,19 @@ fn optimize_plan_node( rule: &dyn OptimizerRule, config: &dyn OptimizerConfig, ) -> Result> { - // TODO: add API to OptimizerRule to allow rewriting by ownership - rule.try_optimize(&plan, config) - .map(|maybe_plan| match maybe_plan { - Some(new_plan) => Transformed::yes(new_plan), + if rule.supports_rewrite() { + return rule.rewrite(plan, config); + } + + rule.try_optimize(&plan, config).map(|maybe_plan| { + match maybe_plan { + Some(new_plan) => { + // if the node was rewritten by the optimizer, replace the node + Transformed::yes(new_plan) + } None => Transformed::no(plan), - }) + } + }) } impl Optimizer { diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 4e06730133d9..17312fa6548a 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -19,12 +19,14 @@ use std::sync::Arc; -use datafusion_common::{DFSchema, DFSchemaRef, Result}; +use datafusion_common::tree_node::Transformed; +use datafusion_common::{internal_err, DFSchema, DFSchemaRef, DataFusionError, Result}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::logical_plan::LogicalPlan; use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::utils::merge_schema; +use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use super::ExprSimplifier; @@ -46,29 +48,47 @@ use super::ExprSimplifier; pub struct SimplifyExpressions {} impl OptimizerRule for SimplifyExpressions { + fn try_optimize( + &self, + _plan: &LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result> { + internal_err!("Should have called SimplifyExpressions::try_optimize_owned") + } + fn name(&self) -> &str { "simplify_expressions" } - fn try_optimize( + fn apply_order(&self) -> Option { + Some(ApplyOrder::BottomUp) + } + + fn supports_rewrite(&self) -> bool { + true + } + + /// if supports_owned returns true, the Optimizer calls + /// [`Self::rewrite`] instead of [`Self::try_optimize`] + fn rewrite( &self, - plan: &LogicalPlan, + plan: LogicalPlan, config: &dyn OptimizerConfig, - ) -> Result> { + ) -> Result, DataFusionError> { let mut execution_props = ExecutionProps::new(); execution_props.query_execution_start_time = config.query_execution_start_time(); - Ok(Some(Self::optimize_internal(plan, &execution_props)?)) + Self::optimize_internal(plan, &execution_props) } } impl SimplifyExpressions { fn optimize_internal( - plan: &LogicalPlan, + plan: LogicalPlan, execution_props: &ExecutionProps, - ) -> Result { + ) -> Result> { let schema = if !plan.inputs().is_empty() { DFSchemaRef::new(merge_schema(plan.inputs())) - } else if let LogicalPlan::TableScan(scan) = plan { + } else if let LogicalPlan::TableScan(scan) = &plan { // When predicates are pushed into a table scan, there is no input // schema to resolve predicates against, so it must be handled specially // @@ -86,13 +106,11 @@ impl SimplifyExpressions { } else { Arc::new(DFSchema::empty()) }; + let info = SimplifyContext::new(execution_props).with_schema(schema); - let new_inputs = plan - .inputs() - .iter() - .map(|input| Self::optimize_internal(input, execution_props)) - .collect::>>()?; + // Inputs have already been rewritten (due to bottom-up traversal handled by Optimizer) + // Just need to rewrite our own expressions let simplifier = ExprSimplifier::new(info); @@ -109,18 +127,22 @@ impl SimplifyExpressions { simplifier }; - let exprs = plan - .expressions() - .into_iter() - .map(|e| { + // the output schema of a filter or join is the input schema. Thus they + // can't handle aliased expressions + let use_alias = !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)); + plan.map_expressions(|e| { + let new_e = if use_alias { // TODO: unify with `rewrite_preserving_name` let original_name = e.name_for_alias()?; - let new_e = simplifier.simplify(e)?; - new_e.alias_if_changed(original_name) - }) - .collect::>>()?; + simplifier.simplify(e)?.alias_if_changed(original_name) + } else { + simplifier.simplify(e) + }?; - plan.with_new_exprs(exprs, new_inputs) + // TODO it would be nice to have a way to know if the expression was simplified + // or not. For now conservatively return Transformed::yes + Ok(Transformed::yes(new_e)) + }) } } @@ -138,6 +160,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use chrono::{DateTime, Utc}; + use crate::optimizer::Optimizer; use datafusion_expr::logical_plan::builder::table_scan_with_filters; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ @@ -165,12 +188,12 @@ mod tests { .expect("building plan") } - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { - let rule = SimplifyExpressions::new(); - let optimized_plan = rule - .try_optimize(plan, &OptimizerContext::new()) - .unwrap() - .expect("failed to optimize plan"); + fn assert_optimized_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { + // Use Optimizer to do plan traversal + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + let optimizer = Optimizer::with_rules(vec![Arc::new(SimplifyExpressions::new())]); + let optimized_plan = + optimizer.optimize(plan, &OptimizerContext::new(), observe)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); Ok(()) @@ -198,7 +221,7 @@ mod tests { let expected = "TableScan: test projection=[a], full_filters=[Boolean(true) AS b IS NOT NULL]"; - assert_optimized_plan_eq(&table_scan, expected) + assert_optimized_plan_eq(table_scan, expected) } #[test] @@ -210,7 +233,7 @@ mod tests { .build()?; assert_optimized_plan_eq( - &plan, + plan, "\ Filter: test.b > Int32(1)\ \n Projection: test.a\ @@ -227,7 +250,7 @@ mod tests { .build()?; assert_optimized_plan_eq( - &plan, + plan, "\ Filter: test.b > Int32(1)\ \n Projection: test.a\ @@ -244,7 +267,7 @@ mod tests { .build()?; assert_optimized_plan_eq( - &plan, + plan, "\ Filter: test.b > Int32(1)\ \n Projection: test.a\ @@ -265,7 +288,7 @@ mod tests { .build()?; assert_optimized_plan_eq( - &plan, + plan, "\ Filter: test.a > Int32(5) AND test.b < Int32(6)\ \n Projection: test.a, test.b\ @@ -288,7 +311,7 @@ mod tests { \n Filter: test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -308,7 +331,7 @@ mod tests { \n Filter: NOT test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -324,7 +347,7 @@ mod tests { \n Filter: NOT test.b AND test.c\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -340,7 +363,7 @@ mod tests { \n Filter: NOT test.b OR NOT test.c\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -356,7 +379,7 @@ mod tests { \n Filter: test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -370,7 +393,7 @@ mod tests { Projection: test.a, test.d, NOT test.b AS test.b = Boolean(false)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -392,7 +415,7 @@ mod tests { \n Projection: test.a, test.c, test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -413,20 +436,17 @@ mod tests { let expected = "\ Values: (Int32(3) AS Int32(1) + Int32(2), Int32(1) AS Int32(2) - Int32(1))"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } fn get_optimized_plan_formatted( - plan: &LogicalPlan, + plan: LogicalPlan, date_time: &DateTime, ) -> String { let config = OptimizerContext::new().with_query_execution_start_time(*date_time); let rule = SimplifyExpressions::new(); - let optimized_plan = rule - .try_optimize(plan, &config) - .unwrap() - .expect("failed to optimize plan"); + let optimized_plan = rule.rewrite(plan, &config).unwrap().data; format!("{optimized_plan:?}") } @@ -440,7 +460,7 @@ mod tests { let expected = "Projection: Int32(0) AS Utf8(\"0\")\ \n TableScan: test"; - let actual = get_optimized_plan_formatted(&plan, &Utc::now()); + let actual = get_optimized_plan_formatted(plan, &Utc::now()); assert_eq!(expected, actual); Ok(()) } @@ -457,7 +477,7 @@ mod tests { .project(proj)? .build()?; - let actual = get_optimized_plan_formatted(&plan, &time); + let actual = get_optimized_plan_formatted(plan, &time); let expected = "Projection: NOT test.a AS Boolean(true) OR Boolean(false) != test.a\ \n TableScan: test"; @@ -476,7 +496,7 @@ mod tests { let expected = "Filter: test.d <= Int32(10)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -489,7 +509,7 @@ mod tests { let expected = "Filter: test.d <= Int32(10) OR test.d >= Int32(100)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -502,7 +522,7 @@ mod tests { let expected = "Filter: test.d <= Int32(10) AND test.d >= Int32(100)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -515,7 +535,7 @@ mod tests { let expected = "Filter: test.d > Int32(10)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -528,7 +548,7 @@ mod tests { let expected = "Filter: test.e IS NOT NULL\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -541,7 +561,7 @@ mod tests { let expected = "Filter: test.e IS NULL\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -555,7 +575,7 @@ mod tests { "Filter: test.d != Int32(1) AND test.d != Int32(2) AND test.d != Int32(3)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -569,7 +589,7 @@ mod tests { "Filter: test.d = Int32(1) OR test.d = Int32(2) OR test.d = Int32(3)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -583,7 +603,7 @@ mod tests { let expected = "Filter: test.d < Int32(1) OR test.d > Int32(10)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -597,7 +617,7 @@ mod tests { let expected = "Filter: test.d >= Int32(1) AND test.d <= Int32(10)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -617,7 +637,7 @@ mod tests { let expected = "Filter: test.a NOT LIKE test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -637,7 +657,7 @@ mod tests { let expected = "Filter: test.a LIKE test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -657,7 +677,7 @@ mod tests { let expected = "Filter: test.a NOT ILIKE test.b\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -670,7 +690,7 @@ mod tests { let expected = "Filter: test.d IS NOT DISTINCT FROM Int32(10)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -683,7 +703,7 @@ mod tests { let expected = "Filter: test.d IS DISTINCT FROM Int32(10)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -709,7 +729,7 @@ mod tests { \n TableScan: t1\ \n TableScan: t2"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -722,7 +742,7 @@ mod tests { let expected = "Filter: Boolean(true)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } #[test] @@ -735,6 +755,6 @@ mod tests { let expected = "Filter: Boolean(false)\ \n TableScan: test"; - assert_optimized_plan_eq(&plan, expected) + assert_optimized_plan_eq(plan, expected) } } From e24b0589112edd65f2652d2bba9766c3cc18bc97 Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Thu, 11 Apr 2024 15:36:02 +0300 Subject: [PATCH 12/39] Fix DistinctCount for timestamps with time zone (#10043) * Fix DistinctCount for timestamps with time zone Preserve the original data type in the aggregation state * Add tests for decimal count distinct --- .../src/aggregate/count_distinct/mod.rs | 42 +++++++++++-------- .../src/aggregate/count_distinct/native.rs | 15 +++++-- .../sqllogictest/test_files/aggregate.slt | 37 +++++++++++++--- .../sqllogictest/test_files/decimal.slt | 11 +++++ 4 files changed, 79 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs b/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs index 9c5605f495ea..ee63945eb249 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs @@ -109,12 +109,14 @@ impl AggregateExpr for DistinctCount { UInt16 => Box::new(PrimitiveDistinctCountAccumulator::::new()), UInt32 => Box::new(PrimitiveDistinctCountAccumulator::::new()), UInt64 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - Decimal128(_, _) => { - Box::new(PrimitiveDistinctCountAccumulator::::new()) - } - Decimal256(_, _) => { - Box::new(PrimitiveDistinctCountAccumulator::::new()) - } + dt @ Decimal128(_, _) => Box::new( + PrimitiveDistinctCountAccumulator::::new() + .with_data_type(dt.clone()), + ), + dt @ Decimal256(_, _) => Box::new( + PrimitiveDistinctCountAccumulator::::new() + .with_data_type(dt.clone()), + ), Date32 => Box::new(PrimitiveDistinctCountAccumulator::::new()), Date64 => Box::new(PrimitiveDistinctCountAccumulator::::new()), @@ -130,18 +132,22 @@ impl AggregateExpr for DistinctCount { Time64(Nanosecond) => { Box::new(PrimitiveDistinctCountAccumulator::::new()) } - Timestamp(Microsecond, _) => Box::new(PrimitiveDistinctCountAccumulator::< - TimestampMicrosecondType, - >::new()), - Timestamp(Millisecond, _) => Box::new(PrimitiveDistinctCountAccumulator::< - TimestampMillisecondType, - >::new()), - Timestamp(Nanosecond, _) => Box::new(PrimitiveDistinctCountAccumulator::< - TimestampNanosecondType, - >::new()), - Timestamp(Second, _) => { - Box::new(PrimitiveDistinctCountAccumulator::::new()) - } + dt @ Timestamp(Microsecond, _) => Box::new( + PrimitiveDistinctCountAccumulator::::new() + .with_data_type(dt.clone()), + ), + dt @ Timestamp(Millisecond, _) => Box::new( + PrimitiveDistinctCountAccumulator::::new() + .with_data_type(dt.clone()), + ), + dt @ Timestamp(Nanosecond, _) => Box::new( + PrimitiveDistinctCountAccumulator::::new() + .with_data_type(dt.clone()), + ), + dt @ Timestamp(Second, _) => Box::new( + PrimitiveDistinctCountAccumulator::::new() + .with_data_type(dt.clone()), + ), Float16 => Box::new(FloatDistinctCountAccumulator::::new()), Float32 => Box::new(FloatDistinctCountAccumulator::::new()), diff --git a/datafusion/physical-expr/src/aggregate/count_distinct/native.rs b/datafusion/physical-expr/src/aggregate/count_distinct/native.rs index a44e8b772e5a..8f3ce8acfe07 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct/native.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct/native.rs @@ -30,6 +30,7 @@ use ahash::RandomState; use arrow::array::ArrayRef; use arrow_array::types::ArrowPrimitiveType; use arrow_array::PrimitiveArray; +use arrow_schema::DataType; use datafusion_common::cast::{as_list_array, as_primitive_array}; use datafusion_common::utils::array_into_list_array; @@ -45,6 +46,7 @@ where T::Native: Eq + Hash, { values: HashSet, + data_type: DataType, } impl PrimitiveDistinctCountAccumulator @@ -55,8 +57,14 @@ where pub(super) fn new() -> Self { Self { values: HashSet::default(), + data_type: T::DATA_TYPE, } } + + pub(super) fn with_data_type(mut self, data_type: DataType) -> Self { + self.data_type = data_type; + self + } } impl Accumulator for PrimitiveDistinctCountAccumulator @@ -65,9 +73,10 @@ where T::Native: Eq + Hash, { fn state(&mut self) -> datafusion_common::Result> { - let arr = Arc::new(PrimitiveArray::::from_iter_values( - self.values.iter().cloned(), - )) as ArrayRef; + let arr = Arc::new( + PrimitiveArray::::from_iter_values(self.values.iter().cloned()) + .with_data_type(self.data_type.clone()), + ); let list = Arc::new(array_into_list_array(arr)); Ok(vec![ScalarValue::List(list)]) } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 4929ab485d6d..966236db2732 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1876,18 +1876,22 @@ select arrow_cast(column1, 'Timestamp(Microsecond, None)') as micros, arrow_cast(column1, 'Timestamp(Millisecond, None)') as millis, arrow_cast(column1, 'Timestamp(Second, None)') as secs, + arrow_cast(column1, 'Timestamp(Nanosecond, Some("UTC"))') as nanos_utc, + arrow_cast(column1, 'Timestamp(Microsecond, Some("UTC"))') as micros_utc, + arrow_cast(column1, 'Timestamp(Millisecond, Some("UTC"))') as millis_utc, + arrow_cast(column1, 'Timestamp(Second, Some("UTC"))') as secs_utc, column2 as names, column3 as tag from t_source; # Demonstate the contents -query PPPPTT +query PPPPPPPPTT select * from t; ---- -2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 X -2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 Row 1 X -NULL NULL NULL NULL Row 2 Y -2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 Y +2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 X +2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123450 2011-12-13T11:13:10.123 2011-12-13T11:13:10 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 X +NULL NULL NULL NULL NULL NULL NULL NULL Row 2 Y +2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 Y # aggregate_timestamps_sum @@ -1933,6 +1937,17 @@ SELECT tag, max(nanos), max(micros), max(millis), max(secs) FROM t GROUP BY tag X 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Y 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 +# aggregate_timestamps_count_distinct_with_tz +query IIII +SELECT count(DISTINCT nanos_utc), count(DISTINCT micros_utc), count(DISTINCT millis_utc), count(DISTINCT secs_utc) FROM t; +---- +3 3 3 3 + +query TIIII +SELECT tag, count(DISTINCT nanos_utc), count(DISTINCT micros_utc), count(DISTINCT millis_utc), count(DISTINCT secs_utc) FROM t GROUP BY tag ORDER BY tag; +---- +X 2 2 2 2 +Y 1 1 1 1 # aggregate_timestamps_avg statement error DataFusion error: Error during planning: No function matches the given name and argument types 'AVG\(Timestamp\(Nanosecond, None\)\)'\. You might need to add explicit type casts\. @@ -2285,6 +2300,18 @@ select c2, avg(c1), arrow_typeof(avg(c1)) from d_table GROUP BY c2 ORDER BY c2 A 110.0045 Decimal128(14, 7) B -100.0045 Decimal128(14, 7) +# aggregate_decimal_count_distinct +query I +select count(DISTINCT cast(c1 AS DECIMAL(10, 2))) from d_table +---- +4 + +query TI +select c2, count(DISTINCT cast(c1 AS DECIMAL(10, 2))) from d_table GROUP BY c2 ORDER BY c2 +---- +A 2 +B 2 + # Use PostgresSQL dialect statement ok set datafusion.sql_parser.dialect = 'Postgres'; diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index c220a5fc9a52..3f75e42d9304 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -720,5 +720,16 @@ select count(*),c1 from decimal256_simple group by c1 order by c1; 4 0.00004 5 0.00005 +query I +select count(DISTINCT cast(c1 AS DECIMAL(42, 4))) from decimal256_simple; +---- +2 + +query BI +select c4, count(DISTINCT cast(c1 AS DECIMAL(42, 4))) from decimal256_simple GROUP BY c4 ORDER BY c4; +---- +false 2 +true 2 + statement ok drop table decimal256_simple; From f55c1d90215614ce531a4103c7dbebf318de1cfd Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 11 Apr 2024 10:59:44 -0400 Subject: [PATCH 13/39] Improve documentation on `LogicalPlan` TreeNode methods (#10037) --- datafusion/expr/src/logical_plan/tree_node.rs | 47 ++++++++++++++----- 1 file changed, 35 insertions(+), 12 deletions(-) diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 1eb9d50277dd..3644f89e8b42 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -451,7 +451,10 @@ macro_rules! handle_transform_recursion_up { impl LogicalPlan { /// Calls `f` on all expressions in the current `LogicalPlan` node. /// - /// Note this does not include expressions in child `LogicalPlan` nodes. + /// # Notes + /// * Similar to [`TreeNode::apply`] but for this node's expressions. + /// * Does not include expressions in input `LogicalPlan` nodes + /// * Visits only the top level expressions (Does not recurse into each expression) pub fn apply_expressions Result>( &self, mut f: F, @@ -541,7 +544,9 @@ impl LogicalPlan { /// /// Returns the current node. /// - /// Note this does not include expressions in child `LogicalPlan` nodes. + /// # Notes + /// * Similar to [`TreeNode::map_children`] but for this node's expressions. + /// * Visits only the top level expressions (Does not recurse into each expression) pub fn map_expressions Result>>( self, mut f: F, @@ -757,7 +762,8 @@ impl LogicalPlan { }) } - /// Visits a plan similarly to [`Self::visit`], but including embedded subqueries. + /// Visits a plan similarly to [`Self::visit`], including subqueries that + /// may appear in expressions such as `IN (SELECT ...)`. pub fn visit_with_subqueries>( &self, visitor: &mut V, @@ -771,7 +777,9 @@ impl LogicalPlan { .visit_parent(|| visitor.f_up(self)) } - /// Rewrites a plan similarly t [`Self::visit`], but including embedded subqueries. + /// Similarly to [`Self::rewrite`], rewrites this node and its inputs using `f`, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn rewrite_with_subqueries>( self, rewriter: &mut R, @@ -783,10 +791,9 @@ impl LogicalPlan { ) } - /// Calls `f` recursively on all children of the `LogicalPlan` node. - /// - /// Unlike [`Self::apply`], this method *does* includes `LogicalPlan`s that - /// are referenced in `Expr`s + /// Similarly to [`Self::apply`], calls `f` on this node and all its inputs, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn apply_with_subqueries Result>( &self, f: &mut F, @@ -796,6 +803,9 @@ impl LogicalPlan { .visit_sibling(|| self.apply_children(|c| c.apply_with_subqueries(f))) } + /// Similarly to [`Self::transform`], rewrites this node and its inputs using `f`, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn transform_with_subqueries Result>>( self, f: &F, @@ -803,6 +813,9 @@ impl LogicalPlan { self.transform_up_with_subqueries(f) } + /// Similarly to [`Self::transform_down`], rewrites this node and its inputs using `f`, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn transform_down_with_subqueries Result>>( self, f: &F, @@ -810,6 +823,9 @@ impl LogicalPlan { handle_transform_recursion_down!(f(self), |c| c.transform_down_with_subqueries(f)) } + /// Similarly to [`Self::transform_down_mut`], rewrites this node and its inputs using `f`, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn transform_down_mut_with_subqueries< F: FnMut(Self) -> Result>, >( @@ -820,6 +836,9 @@ impl LogicalPlan { .transform_down_mut_with_subqueries(f)) } + /// Similarly to [`Self::transform_up`], rewrites this node and its inputs using `f`, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn transform_up_with_subqueries Result>>( self, f: &F, @@ -836,6 +855,9 @@ impl LogicalPlan { handle_transform_recursion_up!(self, |c| c.transform_up_mut_with_subqueries(f), f) } + /// Similarly to [`Self::transform_down`], rewrites this node and its inputs using `f`, + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn transform_down_up_with_subqueries< FD: FnMut(Self) -> Result>, FU: FnMut(Self) -> Result>, @@ -851,8 +873,9 @@ impl LogicalPlan { ) } - /// Calls `f` on all subqueries referenced in expressions of the current - /// `LogicalPlan` node. + /// Similarly to [`Self::apply`], calls `f` on this node and its inputs + /// including subqueries that may appear in expressions such as `IN (SELECT + /// ...)`. pub fn apply_subqueries Result>( &self, mut f: F, @@ -872,8 +895,8 @@ impl LogicalPlan { }) } - /// Rewrites all subquery `LogicalPlan` in the current `LogicalPlan` node - /// using `f`. + /// Similarly to [`Self::map_children`], rewrites all subqueries that may + /// appear in expressions such as `IN (SELECT ...)` using `f`. /// /// Returns the current node. pub fn map_subqueries Result>>( From 7dc20b81de8cf50bdb2b90e5f216a6324b79e340 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Thu, 11 Apr 2024 18:22:01 +0200 Subject: [PATCH 14/39] chore(deps): update prost-build requirement from =0.12.3 to =0.12.4 (#10045) * chore(deps): update prost-build requirement from =0.12.3 to =0.12.4 Updates the requirements on [prost-build](https://github.com/tokio-rs/prost) to permit the latest version. - [Release notes](https://github.com/tokio-rs/prost/releases) - [Commits](https://github.com/tokio-rs/prost/compare/v0.12.3...v0.12.4) --- updated-dependencies: - dependency-name: prost-build dependency-type: direct:production ... Signed-off-by: dependabot[bot] * re-gen protobuf code --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/proto/gen/Cargo.toml | 2 +- datafusion/proto/src/generated/prost.rs | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/proto/gen/Cargo.toml b/datafusion/proto/gen/Cargo.toml index e843827a91ac..01ce92ee9e8e 100644 --- a/datafusion/proto/gen/Cargo.toml +++ b/datafusion/proto/gen/Cargo.toml @@ -32,4 +32,4 @@ publish = false [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.6.2" -prost-build = "=0.12.3" +prost-build = "=0.12.4" diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 042c794e19de..d6a27dbc5652 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1,3 +1,4 @@ +// This file is @generated by prost-build. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ColumnRelation { From feb9100432d453f19d40428265d2aa9a9f942d5d Mon Sep 17 00:00:00 2001 From: colommar Date: Fri, 12 Apr 2024 02:55:25 +0800 Subject: [PATCH 15/39] fix#9501 (#10028) --- benchmarks/queries/clickbench/README.md | 4 +- datafusion-cli/src/exec.rs | 2 +- .../physical_plan/parquet/row_groups.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 2 +- datafusion/optimizer/README.md | 10 +- docs/source/library-user-guide/adding-udfs.md | 2 +- docs/source/user-guide/cli.md | 20 ++-- docs/source/user-guide/example-usage.md | 2 +- .../source/user-guide/sql/scalar_functions.md | 102 +++++++++--------- 9 files changed, 73 insertions(+), 73 deletions(-) diff --git a/benchmarks/queries/clickbench/README.md b/benchmarks/queries/clickbench/README.md index ef540ccf9c91..29b1a7588f17 100644 --- a/benchmarks/queries/clickbench/README.md +++ b/benchmarks/queries/clickbench/README.md @@ -63,7 +63,7 @@ LIMIT 10; Here are some interesting statistics about the data used in the queries Max length of `"SearchPhrase"` is 1113 characters ```sql -❯ select min(length("SearchPhrase")) as "SearchPhrase_len_min", max(length("SearchPhrase")) "SearchPhrase_len_max" from 'hits.parquet' limit 10; +> select min(length("SearchPhrase")) as "SearchPhrase_len_min", max(length("SearchPhrase")) "SearchPhrase_len_max" from 'hits.parquet' limit 10; +----------------------+----------------------+ | SearchPhrase_len_min | SearchPhrase_len_max | +----------------------+----------------------+ @@ -74,7 +74,7 @@ Max length of `"SearchPhrase"` is 1113 characters Here is the schema of the data ```sql -❯ describe 'hits.parquet'; +> describe 'hits.parquet'; +-----------------------+-----------+-------------+ | column_name | data_type | is_nullable | +-----------------------+-----------+-------------+ diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index be7e5275e8b2..2072cc7df002 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -131,7 +131,7 @@ pub async fn exec_from_repl( rl.load_history(".history").ok(); loop { - match rl.readline("❯ ") { + match rl.readline("> ") { Ok(line) if line.starts_with('\\') => { rl.add_history_entry(line.trim_end())?; let command = line.split_whitespace().collect::>().join(" "); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 2b9665954842..a4dfd9b96870 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -1212,7 +1212,7 @@ mod tests { /// Return a test for data_index_bloom_encoding_stats.parquet /// Note the values in the `String` column are: /// ```sql - /// ❯ select * from './parquet-testing/data/data_index_bloom_encoding_stats.parquet'; + /// > select * from './parquet-testing/data/data_index_bloom_encoding_stats.parquet'; /// +-----------+ /// | String | /// +-----------+ diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d16dfb140353..cb8c97c71e38 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2127,7 +2127,7 @@ pub struct Prepare { /// # Example output: /// /// ```sql -/// ❯ describe traces; +/// > describe traces; /// +--------------------+-----------------------------+-------------+ /// | column_name | data_type | is_nullable | /// +--------------------+-----------------------------+-------------+ diff --git a/datafusion/optimizer/README.md b/datafusion/optimizer/README.md index 4f9e0fb98526..b0f4c5a72014 100644 --- a/datafusion/optimizer/README.md +++ b/datafusion/optimizer/README.md @@ -104,7 +104,7 @@ Every expression in DataFusion has a name, which is used as the column name. For contains a single column with the name `"COUNT(aggregate_test_100.c9)"`: ```text -❯ select count(c9) from aggregate_test_100; +> select count(c9) from aggregate_test_100; +------------------------------+ | COUNT(aggregate_test_100.c9) | +------------------------------+ @@ -116,7 +116,7 @@ These names are used to refer to the columns in both subqueries as well as inter to another. For example: ```text -❯ select "COUNT(aggregate_test_100.c9)" + 1 from (select count(c9) from aggregate_test_100) as sq; +> select "COUNT(aggregate_test_100.c9)" + 1 from (select count(c9) from aggregate_test_100) as sq; +--------------------------------------------+ | sq.COUNT(aggregate_test_100.c9) + Int64(1) | +--------------------------------------------+ @@ -134,7 +134,7 @@ Here is a simple example of such a rewrite. The expression `1 + 2` can be intern displayed the same as `1 + 2`: ```text -❯ select 1 + 2; +> select 1 + 2; +---------------------+ | Int64(1) + Int64(2) | +---------------------+ @@ -146,7 +146,7 @@ Looking at the `EXPLAIN` output we can see that the optimizer has effectively re `3 as "1 + 2"`: ```text -❯ explain select 1 + 2; +> explain select 1 + 2; +---------------+-------------------------------------------------+ | plan_type | plan | +---------------+-------------------------------------------------+ @@ -289,7 +289,7 @@ The `EXPLAIN VERBOSE` command can be used to show the effect of each optimizatio In the following example, the `type_coercion` and `simplify_expressions` passes have simplified the plan so that it returns the constant `"3.2"` rather than doing a computation at execution time. ```text -❯ explain verbose select cast(1 + 2.2 as string) as foo; +> explain verbose select cast(1 + 2.2 as string) as foo; +------------------------------------------------------------+---------------------------------------------------------------------------+ | plan_type | plan | +------------------------------------------------------------+---------------------------------------------------------------------------+ diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index ad210724103d..653c1f9d3784 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -536,7 +536,7 @@ Because we're returning a `TableProvider`, in this example we'll use the `MemTab While this is a simple example for illustrative purposes, UDTFs have a lot of potential use cases. And can be particularly useful for reading data from external sources and interactive analysis. For example, see the [example][4] for a working example that reads from a CSV file. As another example, you could use the built-in UDTF `parquet_metadata` in the CLI to read the metadata from a Parquet file. ```console -❯ select filename, row_group_id, row_group_num_rows, row_group_bytes, stats_min, stats_max from parquet_metadata('./benchmarks/data/hits.parquet') where column_id = 17 limit 10; +> select filename, row_group_id, row_group_num_rows, row_group_bytes, stats_min, stats_max from parquet_metadata('./benchmarks/data/hits.parquet') where column_id = 17 limit 10; +--------------------------------+--------------+--------------------+-----------------+-----------+-----------+ | filename | row_group_id | row_group_num_rows | row_group_bytes | stats_min | stats_max | +--------------------------------+--------------+--------------------+-----------------+-----------+-----------+ diff --git a/docs/source/user-guide/cli.md b/docs/source/user-guide/cli.md index da4c9870545a..9c3fc8bd60c8 100644 --- a/docs/source/user-guide/cli.md +++ b/docs/source/user-guide/cli.md @@ -165,7 +165,7 @@ Query that single file (the CLI also supports parquet, compressed csv, avro, jso ```shell $ datafusion-cli DataFusion CLI v17.0.0 -❯ select * from 'data.csv'; +> select * from 'data.csv'; +---+---+ | a | b | +---+---+ @@ -184,7 +184,7 @@ data.csv data2.csv ```shell $ datafusion-cli DataFusion CLI v16.0.0 -❯ select * from 'data_dir'; +> select * from 'data_dir'; +---+---+ | a | b | +---+---+ @@ -335,9 +335,9 @@ $ export AWS_ACCESS_KEY_ID=****** $ datafusion-cli DataFusion CLI v21.0.0 -❯ create external table test stored as parquet location 's3://bucket/path/file.parquet'; +> create external table test stored as parquet location 's3://bucket/path/file.parquet'; 0 rows in set. Query took 0.374 seconds. -❯ select * from test; +> select * from test; +----------+----------+ | column_1 | column_2 | +----------+----------+ @@ -429,9 +429,9 @@ $ export GOOGLE_SERVICE_ACCOUNT=/tmp/gcs.json $ datafusion-cli DataFusion CLI v21.0.0 -❯ create external table test stored as parquet location 'gs://bucket/path/file.parquet'; +> create external table test stored as parquet location 'gs://bucket/path/file.parquet'; 0 rows in set. Query took 0.374 seconds. -❯ select * from test; +> select * from test; +----------+----------+ | column_1 | column_2 | +----------+----------+ @@ -619,7 +619,7 @@ appropriately: ```shell $ DATAFUSION_EXECUTION_BATCH_SIZE=1024 datafusion-cli DataFusion CLI v12.0.0 -❯ show all; +> show all; +-------------------------------------------------+---------+ | name | value | +-------------------------------------------------+---------+ @@ -639,7 +639,7 @@ You can change the configuration options using `SET` statement as well ```shell $ datafusion-cli DataFusion CLI v13.0.0 -❯ show datafusion.execution.batch_size; +> show datafusion.execution.batch_size; +---------------------------------+---------+ | name | value | +---------------------------------+---------+ @@ -647,10 +647,10 @@ DataFusion CLI v13.0.0 +---------------------------------+---------+ 1 row in set. Query took 0.011 seconds. -❯ set datafusion.execution.batch_size to 1024; +> set datafusion.execution.batch_size to 1024; 0 rows in set. Query took 0.000 seconds. -❯ show datafusion.execution.batch_size; +> show datafusion.execution.batch_size; +---------------------------------+---------+ | name | value | +---------------------------------+---------+ diff --git a/docs/source/user-guide/example-usage.md b/docs/source/user-guide/example-usage.md index 31b599ac3308..6e4bf68fa018 100644 --- a/docs/source/user-guide/example-usage.md +++ b/docs/source/user-guide/example-usage.md @@ -261,7 +261,7 @@ Set environment [variables](https://doc.rust-lang.org/std/backtrace/index.html#e ```bash RUST_BACKTRACE=1 ./target/debug/datafusion-cli DataFusion CLI v31.0.0 -❯ select row_numer() over (partition by a order by a) from (select 1 a); +> select row_numer() over (partition by a order by a) from (select 1 a); Error during planning: Invalid function 'row_numer'. Did you mean 'ROW_NUMBER'? diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 62b81ea7ea4b..217bd5f05a86 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1650,13 +1650,13 @@ make_date(year, month, day) #### Example ``` -❯ select make_date(2023, 1, 31); +> select make_date(2023, 1, 31); +-------------------------------------------+ | make_date(Int64(2023),Int64(1),Int64(31)) | +-------------------------------------------+ | 2023-01-31 | +-------------------------------------------+ -❯ select make_date('2023', '01', '31'); +> select make_date('2023', '01', '31'); +-----------------------------------------------+ | make_date(Utf8("2023"),Utf8("01"),Utf8("31")) | +-----------------------------------------------+ @@ -1686,7 +1686,7 @@ to_char(expression, format) #### Example ``` -❯ ❯ select to_char('2023-03-01'::date, '%d-%m-%Y'); +> > select to_char('2023-03-01'::date, '%d-%m-%Y'); +----------------------------------------------+ | to_char(Utf8("2023-03-01"),Utf8("%d-%m-%Y")) | +----------------------------------------------+ @@ -1731,13 +1731,13 @@ to_timestamp(expression[, ..., format_n]) #### Example ``` -❯ select to_timestamp('2023-01-31T09:26:56.123456789-05:00'); +> select to_timestamp('2023-01-31T09:26:56.123456789-05:00'); +-----------------------------------------------------------+ | to_timestamp(Utf8("2023-01-31T09:26:56.123456789-05:00")) | +-----------------------------------------------------------+ | 2023-01-31T14:26:56.123456789 | +-----------------------------------------------------------+ -❯ select to_timestamp('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +> select to_timestamp('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +--------------------------------------------------------------------------------------------------------+ | to_timestamp(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | +--------------------------------------------------------------------------------------------------------+ @@ -1770,13 +1770,13 @@ to_timestamp_millis(expression[, ..., format_n]) #### Example ``` -❯ select to_timestamp_millis('2023-01-31T09:26:56.123456789-05:00'); +> select to_timestamp_millis('2023-01-31T09:26:56.123456789-05:00'); +------------------------------------------------------------------+ | to_timestamp_millis(Utf8("2023-01-31T09:26:56.123456789-05:00")) | +------------------------------------------------------------------+ | 2023-01-31T14:26:56.123 | +------------------------------------------------------------------+ -❯ select to_timestamp_millis('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +> select to_timestamp_millis('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +---------------------------------------------------------------------------------------------------------------+ | to_timestamp_millis(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | +---------------------------------------------------------------------------------------------------------------+ @@ -1809,13 +1809,13 @@ to_timestamp_micros(expression[, ..., format_n]) #### Example ``` -❯ select to_timestamp_micros('2023-01-31T09:26:56.123456789-05:00'); +> select to_timestamp_micros('2023-01-31T09:26:56.123456789-05:00'); +------------------------------------------------------------------+ | to_timestamp_micros(Utf8("2023-01-31T09:26:56.123456789-05:00")) | +------------------------------------------------------------------+ | 2023-01-31T14:26:56.123456 | +------------------------------------------------------------------+ -❯ select to_timestamp_micros('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +> select to_timestamp_micros('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +---------------------------------------------------------------------------------------------------------------+ | to_timestamp_micros(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | +---------------------------------------------------------------------------------------------------------------+ @@ -1848,13 +1848,13 @@ to_timestamp_nanos(expression[, ..., format_n]) #### Example ``` -❯ select to_timestamp_nanos('2023-01-31T09:26:56.123456789-05:00'); +> select to_timestamp_nanos('2023-01-31T09:26:56.123456789-05:00'); +-----------------------------------------------------------------+ | to_timestamp_nanos(Utf8("2023-01-31T09:26:56.123456789-05:00")) | +-----------------------------------------------------------------+ | 2023-01-31T14:26:56.123456789 | +-----------------------------------------------------------------+ -❯ select to_timestamp_nanos('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +> select to_timestamp_nanos('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +--------------------------------------------------------------------------------------------------------------+ | to_timestamp_nanos(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | +--------------------------------------------------------------------------------------------------------------+ @@ -1887,13 +1887,13 @@ to_timestamp_seconds(expression[, ..., format_n]) #### Example ``` -❯ select to_timestamp_seconds('2023-01-31T09:26:56.123456789-05:00'); +> select to_timestamp_seconds('2023-01-31T09:26:56.123456789-05:00'); +-------------------------------------------------------------------+ | to_timestamp_seconds(Utf8("2023-01-31T09:26:56.123456789-05:00")) | +-------------------------------------------------------------------+ | 2023-01-31T14:26:56 | +-------------------------------------------------------------------+ -❯ select to_timestamp_seconds('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +> select to_timestamp_seconds('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); +----------------------------------------------------------------------------------------------------------------+ | to_timestamp_seconds(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | +----------------------------------------------------------------------------------------------------------------+ @@ -2022,7 +2022,7 @@ array_append(array, element) #### Example ``` -❯ select array_append([1, 2, 3], 4); +> select array_append([1, 2, 3], 4); +--------------------------------------+ | array_append(List([1,2,3]),Int64(4)) | +--------------------------------------+ @@ -2054,7 +2054,7 @@ array_sort(array, desc, nulls_first) #### Example ``` -❯ select array_sort([3, 1, 2]); +> select array_sort([3, 1, 2]); +-----------------------------+ | array_sort(List([3,1,2])) | +-----------------------------+ @@ -2084,7 +2084,7 @@ array_resize(array, size, value) #### Example ``` -❯ select array_resize([1, 2, 3], 5, 0); +> select array_resize([1, 2, 3], 5, 0); +-------------------------------------+ | array_resize(List([1,2,3],5,0)) | +-------------------------------------+ @@ -2117,7 +2117,7 @@ array_concat(array[, ..., array_n]) #### Example ``` -❯ select array_concat([1, 2], [3, 4], [5, 6]); +> select array_concat([1, 2], [3, 4], [5, 6]); +---------------------------------------------------+ | array_concat(List([1,2]),List([3,4]),List([5,6])) | +---------------------------------------------------+ @@ -2208,7 +2208,7 @@ array_dims(array) #### Example ``` -❯ select array_dims([[1, 2, 3], [4, 5, 6]]); +> select array_dims([[1, 2, 3], [4, 5, 6]]); +---------------------------------+ | array_dims(List([1,2,3,4,5,6])) | +---------------------------------+ @@ -2236,7 +2236,7 @@ array_distinct(array) #### Example ``` -❯ select array_distinct([1, 3, 2, 3, 1, 2, 4]); +> select array_distinct([1, 3, 2, 3, 1, 2, 4]); +---------------------------------+ | array_distinct(List([1,2,3,4])) | +---------------------------------+ @@ -2265,7 +2265,7 @@ array_element(array, index) #### Example ``` -❯ select array_element([1, 2, 3, 4], 3); +> select array_element([1, 2, 3, 4], 3); +-----------------------------------------+ | array_element(List([1,2,3,4]),Int64(3)) | +-----------------------------------------+ @@ -2339,13 +2339,13 @@ array_intersect(array1, array2) #### Example ``` -❯ select array_intersect([1, 2, 3, 4], [5, 6, 3, 4]); +> select array_intersect([1, 2, 3, 4], [5, 6, 3, 4]); +----------------------------------------------------+ | array_intersect([1, 2, 3, 4], [5, 6, 3, 4]); | +----------------------------------------------------+ | [3, 4] | +----------------------------------------------------+ -❯ select array_intersect([1, 2, 3, 4], [5, 6, 7, 8]); +> select array_intersect([1, 2, 3, 4], [5, 6, 7, 8]); +----------------------------------------------------+ | array_intersect([1, 2, 3, 4], [5, 6, 7, 8]); | +----------------------------------------------------+ @@ -2380,7 +2380,7 @@ array_length(array, dimension) #### Example ``` -❯ select array_length([1, 2, 3, 4, 5]); +> select array_length([1, 2, 3, 4, 5]); +---------------------------------+ | array_length(List([1,2,3,4,5])) | +---------------------------------+ @@ -2408,7 +2408,7 @@ array_ndims(array, element) #### Example ``` -❯ select array_ndims([[1, 2, 3], [4, 5, 6]]); +> select array_ndims([[1, 2, 3], [4, 5, 6]]); +----------------------------------+ | array_ndims(List([1,2,3,4,5,6])) | +----------------------------------+ @@ -2437,7 +2437,7 @@ array_prepend(element, array) #### Example ``` -❯ select array_prepend(1, [2, 3, 4]); +> select array_prepend(1, [2, 3, 4]); +---------------------------------------+ | array_prepend(Int64(1),List([2,3,4])) | +---------------------------------------+ @@ -2467,7 +2467,7 @@ array_pop_front(array) #### Example ``` -❯ select array_pop_front([1, 2, 3]); +> select array_pop_front([1, 2, 3]); +-------------------------------+ | array_pop_front(List([1,2,3])) | +-------------------------------+ @@ -2495,7 +2495,7 @@ array_pop_back(array) #### Example ``` -❯ select array_pop_back([1, 2, 3]); +> select array_pop_back([1, 2, 3]); +-------------------------------+ | array_pop_back(List([1,2,3])) | +-------------------------------+ @@ -2526,7 +2526,7 @@ array_position(array, element, index) #### Example ``` -❯ select array_position([1, 2, 2, 3, 1, 4], 2); +> select array_position([1, 2, 2, 3, 1, 4], 2); +----------------------------------------------+ | array_position(List([1,2,2,3,1,4]),Int64(2)) | +----------------------------------------------+ @@ -2557,7 +2557,7 @@ array_positions(array, element) #### Example ``` -❯ select array_positions([1, 2, 2, 3, 1, 4], 2); +> select array_positions([1, 2, 2, 3, 1, 4], 2); +-----------------------------------------------+ | array_positions(List([1,2,2,3,1,4]),Int64(2)) | +-----------------------------------------------+ @@ -2594,7 +2594,7 @@ array_repeat(element, count) #### Example ``` -❯ select array_repeat(1, 3); +> select array_repeat(1, 3); +---------------------------------+ | array_repeat(Int64(1),Int64(3)) | +---------------------------------+ @@ -2603,7 +2603,7 @@ array_repeat(element, count) ``` ``` -❯ select array_repeat([1, 2], 2); +> select array_repeat([1, 2], 2); +------------------------------------+ | array_repeat(List([1,2]),Int64(2)) | +------------------------------------+ @@ -2632,7 +2632,7 @@ array_remove(array, element) #### Example ``` -❯ select array_remove([1, 2, 2, 3, 2, 1, 4], 2); +> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); +----------------------------------------------+ | array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | +----------------------------------------------+ @@ -2662,7 +2662,7 @@ array_remove_n(array, element, max) #### Example ``` -❯ select array_remove_n([1, 2, 2, 3, 2, 1, 4], 2, 2); +> select array_remove_n([1, 2, 2, 3, 2, 1, 4], 2, 2); +---------------------------------------------------------+ | array_remove_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(2)) | +---------------------------------------------------------+ @@ -2691,7 +2691,7 @@ array_remove_all(array, element) #### Example ``` -❯ select array_remove_all([1, 2, 2, 3, 2, 1, 4], 2); +> select array_remove_all([1, 2, 2, 3, 2, 1, 4], 2); +--------------------------------------------------+ | array_remove_all(List([1,2,2,3,2,1,4]),Int64(2)) | +--------------------------------------------------+ @@ -2721,7 +2721,7 @@ array_replace(array, from, to) #### Example ``` -❯ select array_replace([1, 2, 2, 3, 2, 1, 4], 2, 5); +> select array_replace([1, 2, 2, 3, 2, 1, 4], 2, 5); +--------------------------------------------------------+ | array_replace(List([1,2,2,3,2,1,4]),Int64(2),Int64(5)) | +--------------------------------------------------------+ @@ -2752,7 +2752,7 @@ array_replace_n(array, from, to, max) #### Example ``` -❯ select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); +> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); +-------------------------------------------------------------------+ | array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | +-------------------------------------------------------------------+ @@ -2782,7 +2782,7 @@ array_replace_all(array, from, to) #### Example ``` -❯ select array_replace_all([1, 2, 2, 3, 2, 1, 4], 2, 5); +> select array_replace_all([1, 2, 2, 3, 2, 1, 4], 2, 5); +------------------------------------------------------------+ | array_replace_all(List([1,2,2,3,2,1,4]),Int64(2),Int64(5)) | +------------------------------------------------------------+ @@ -2810,7 +2810,7 @@ array_reverse(array) #### Example ``` -❯ select array_reverse([1, 2, 3, 4]); +> select array_reverse([1, 2, 3, 4]); +------------------------------------------------------------+ | array_reverse(List([1, 2, 3, 4])) | +------------------------------------------------------------+ @@ -2843,7 +2843,7 @@ array_slice(array, begin, end) #### Example ``` -❯ select array_slice([1, 2, 3, 4, 5, 6, 7, 8], 3, 6); +> select array_slice([1, 2, 3, 4, 5, 6, 7, 8], 3, 6); +--------------------------------------------------------+ | array_slice(List([1,2,3,4,5,6,7,8]),Int64(3),Int64(6)) | +--------------------------------------------------------+ @@ -2872,7 +2872,7 @@ array_to_string(array, delimiter) #### Example ``` -❯ select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); +> select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); +----------------------------------------------------+ | array_to_string(List([1,2,3,4,5,6,7,8]),Utf8(",")) | +----------------------------------------------------+ @@ -2904,13 +2904,13 @@ array_union(array1, array2) #### Example ``` -❯ select array_union([1, 2, 3, 4], [5, 6, 3, 4]); +> select array_union([1, 2, 3, 4], [5, 6, 3, 4]); +----------------------------------------------------+ | array_union([1, 2, 3, 4], [5, 6, 3, 4]); | +----------------------------------------------------+ | [1, 2, 3, 4, 5, 6] | +----------------------------------------------------+ -❯ select array_union([1, 2, 3, 4], [5, 6, 7, 8]); +> select array_union([1, 2, 3, 4], [5, 6, 7, 8]); +----------------------------------------------------+ | array_union([1, 2, 3, 4], [5, 6, 7, 8]); | +----------------------------------------------------+ @@ -2942,13 +2942,13 @@ array_except(array1, array2) #### Example ``` -❯ select array_except([1, 2, 3, 4], [5, 6, 3, 4]); +> select array_except([1, 2, 3, 4], [5, 6, 3, 4]); +----------------------------------------------------+ | array_except([1, 2, 3, 4], [5, 6, 3, 4]); | +----------------------------------------------------+ | [1, 2] | +----------------------------------------------------+ -❯ select array_except([1, 2, 3, 4], [3, 4, 5, 6]); +> select array_except([1, 2, 3, 4], [3, 4, 5, 6]); +----------------------------------------------------+ | array_except([1, 2, 3, 4], [3, 4, 5, 6]); | +----------------------------------------------------+ @@ -2978,7 +2978,7 @@ cardinality(array) #### Example ``` -❯ select cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]); +> select cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]); +--------------------------------------+ | cardinality(List([1,2,3,4,5,6,7,8])) | +--------------------------------------+ @@ -3002,7 +3002,7 @@ empty(array) #### Example ``` -❯ select empty([1]); +> select empty([1]); +------------------+ | empty(List([1])) | +------------------+ @@ -3032,7 +3032,7 @@ generate_series(start, stop, step) #### Example ``` -❯ select generate_series(1,3); +> select generate_series(1,3); +------------------------------------+ | generate_series(Int64(1),Int64(3)) | +------------------------------------+ @@ -3209,7 +3209,7 @@ _Alias of [empty](#empty)._ #### Example ``` -❯ select make_array(1, 2, 3, 4, 5); +> select make_array(1, 2, 3, 4, 5); +----------------------------------------------------------+ | make_array(Int64(1),Int64(2),Int64(3),Int64(4),Int64(5)) | +----------------------------------------------------------+ @@ -3337,7 +3337,7 @@ select * from t; +---+---+ -- use default names `c0`, `c1` -❯ select struct(a, b) from t; +> select struct(a, b) from t; +-----------------+ | struct(t.a,t.b) | +-----------------+ @@ -3520,7 +3520,7 @@ arrow_cast(expression, datatype) #### Example ``` -❯ select arrow_cast(-5, 'Int8') as a, +> select arrow_cast(-5, 'Int8') as a, arrow_cast('foo', 'Dictionary(Int32, Utf8)') as b, arrow_cast('bar', 'LargeUtf8') as c, arrow_cast('2023-01-02T12:53:02', 'Timestamp(Microsecond, Some("+08:00"))') as d @@ -3550,7 +3550,7 @@ arrow_typeof(expression) #### Example ``` -❯ select arrow_typeof('foo'), arrow_typeof(1); +> select arrow_typeof('foo'), arrow_typeof(1); +---------------------------+------------------------+ | arrow_typeof(Utf8("foo")) | arrow_typeof(Int64(1)) | +---------------------------+------------------------+ From 118eecdc8384805cae752dac0c4ccc768cc9629b Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Thu, 11 Apr 2024 22:41:46 +0300 Subject: [PATCH 16/39] Always pass DataType to PrimitiveDistinctCountAccumulator (#10047) --- .../src/aggregate/count_distinct/mod.rs | 103 ++++++++++-------- .../src/aggregate/count_distinct/native.rs | 9 +- 2 files changed, 61 insertions(+), 51 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs b/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs index ee63945eb249..ae3370df723a 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct/mod.rs @@ -99,55 +99,70 @@ impl AggregateExpr for DistinctCount { use DataType::*; use TimeUnit::*; - Ok(match &self.state_data_type { + let data_type = &self.state_data_type; + Ok(match data_type { // try and use a specialized accumulator if possible, otherwise fall back to generic accumulator - Int8 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - Int16 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - Int32 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - Int64 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - UInt8 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - UInt16 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - UInt32 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - UInt64 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - dt @ Decimal128(_, _) => Box::new( - PrimitiveDistinctCountAccumulator::::new() - .with_data_type(dt.clone()), - ), - dt @ Decimal256(_, _) => Box::new( - PrimitiveDistinctCountAccumulator::::new() - .with_data_type(dt.clone()), - ), - - Date32 => Box::new(PrimitiveDistinctCountAccumulator::::new()), - Date64 => Box::new(PrimitiveDistinctCountAccumulator::::new()), + Int8 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + Int16 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + Int32 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + Int64 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + UInt8 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + UInt16 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + UInt32 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + UInt64 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + Decimal128(_, _) => Box::new(PrimitiveDistinctCountAccumulator::< + Decimal128Type, + >::new(data_type)), + Decimal256(_, _) => Box::new(PrimitiveDistinctCountAccumulator::< + Decimal256Type, + >::new(data_type)), + + Date32 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), + Date64 => Box::new(PrimitiveDistinctCountAccumulator::::new( + data_type, + )), Time32(Millisecond) => Box::new(PrimitiveDistinctCountAccumulator::< Time32MillisecondType, - >::new()), - Time32(Second) => { - Box::new(PrimitiveDistinctCountAccumulator::::new()) - } + >::new(data_type)), + Time32(Second) => Box::new(PrimitiveDistinctCountAccumulator::< + Time32SecondType, + >::new(data_type)), Time64(Microsecond) => Box::new(PrimitiveDistinctCountAccumulator::< Time64MicrosecondType, - >::new()), - Time64(Nanosecond) => { - Box::new(PrimitiveDistinctCountAccumulator::::new()) - } - dt @ Timestamp(Microsecond, _) => Box::new( - PrimitiveDistinctCountAccumulator::::new() - .with_data_type(dt.clone()), - ), - dt @ Timestamp(Millisecond, _) => Box::new( - PrimitiveDistinctCountAccumulator::::new() - .with_data_type(dt.clone()), - ), - dt @ Timestamp(Nanosecond, _) => Box::new( - PrimitiveDistinctCountAccumulator::::new() - .with_data_type(dt.clone()), - ), - dt @ Timestamp(Second, _) => Box::new( - PrimitiveDistinctCountAccumulator::::new() - .with_data_type(dt.clone()), - ), + >::new(data_type)), + Time64(Nanosecond) => Box::new(PrimitiveDistinctCountAccumulator::< + Time64NanosecondType, + >::new(data_type)), + Timestamp(Microsecond, _) => Box::new(PrimitiveDistinctCountAccumulator::< + TimestampMicrosecondType, + >::new(data_type)), + Timestamp(Millisecond, _) => Box::new(PrimitiveDistinctCountAccumulator::< + TimestampMillisecondType, + >::new(data_type)), + Timestamp(Nanosecond, _) => Box::new(PrimitiveDistinctCountAccumulator::< + TimestampNanosecondType, + >::new(data_type)), + Timestamp(Second, _) => Box::new(PrimitiveDistinctCountAccumulator::< + TimestampSecondType, + >::new(data_type)), Float16 => Box::new(FloatDistinctCountAccumulator::::new()), Float32 => Box::new(FloatDistinctCountAccumulator::::new()), diff --git a/datafusion/physical-expr/src/aggregate/count_distinct/native.rs b/datafusion/physical-expr/src/aggregate/count_distinct/native.rs index 8f3ce8acfe07..97ff1ef257b4 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct/native.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct/native.rs @@ -54,17 +54,12 @@ where T: ArrowPrimitiveType + Send, T::Native: Eq + Hash, { - pub(super) fn new() -> Self { + pub(super) fn new(data_type: &DataType) -> Self { Self { values: HashSet::default(), - data_type: T::DATA_TYPE, + data_type: data_type.clone(), } } - - pub(super) fn with_data_type(mut self, data_type: DataType) -> Self { - self.data_type = data_type; - self - } } impl Accumulator for PrimitiveDistinctCountAccumulator From 2def10ff1a1ad6b99a1ae40692115b28c3efdfc2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 12 Apr 2024 07:21:02 -0400 Subject: [PATCH 17/39] Stop copying plans in `LogicalPlan::with_param_values` (#10016) --- datafusion/expr/src/logical_plan/plan.rs | 79 +++++++++--------------- 1 file changed, 28 insertions(+), 51 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index cb8c97c71e38..91c8670f3805 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -913,14 +913,19 @@ impl LogicalPlan { param_values: impl Into, ) -> Result { let param_values = param_values.into(); - match self { - LogicalPlan::Prepare(prepare_lp) => { - param_values.verify(&prepare_lp.data_types)?; - let input_plan = prepare_lp.input; - input_plan.replace_params_with_values(¶m_values) + let plan_with_values = self.replace_params_with_values(¶m_values)?; + + // unwrap Prepare + Ok(if let LogicalPlan::Prepare(prepare_lp) = plan_with_values { + param_values.verify(&prepare_lp.data_types)?; + // try and take ownership of the input if is not shared, clone otherwise + match Arc::try_unwrap(prepare_lp.input) { + Ok(input) => input, + Err(arc_input) => arc_input.as_ref().clone(), } - _ => self.replace_params_with_values(¶m_values), - } + } else { + plan_with_values + }) } /// Returns the maximum number of rows that this plan can output, if known. @@ -1046,27 +1051,26 @@ impl LogicalPlan { /// ...) replaced with corresponding values provided in /// `params_values` /// - /// See [`Self::with_param_values`] for examples and usage + /// See [`Self::with_param_values`] for examples and usage with an owned + /// `ParamValues` pub fn replace_params_with_values( - &self, + self, param_values: &ParamValues, ) -> Result { - let new_exprs = self - .expressions() - .into_iter() - .map(|e| { - let e = e.infer_placeholder_types(self.schema())?; - Self::replace_placeholders_with_values(e, param_values) + self.transform_up_with_subqueries(&|plan| { + let schema = plan.schema().clone(); + plan.map_expressions(|e| { + e.infer_placeholder_types(&schema)?.transform_up(&|e| { + if let Expr::Placeholder(Placeholder { id, .. }) = e { + let value = param_values.get_placeholders_with_values(&id)?; + Ok(Transformed::yes(Expr::Literal(value))) + } else { + Ok(Transformed::no(e)) + } + }) }) - .collect::>>()?; - - let new_inputs_with_values = self - .inputs() - .into_iter() - .map(|inp| inp.replace_params_with_values(param_values)) - .collect::>>()?; - - self.with_new_exprs(new_exprs, new_inputs_with_values) + }) + .map(|res| res.data) } /// Walk the logical plan, find any `Placeholder` tokens, and return a map of their IDs and DataTypes @@ -1099,33 +1103,6 @@ impl LogicalPlan { .map(|_| param_types) } - /// Return an Expr with all placeholders replaced with their - /// corresponding values provided in the params_values - fn replace_placeholders_with_values( - expr: Expr, - param_values: &ParamValues, - ) -> Result { - expr.transform(&|expr| { - match &expr { - Expr::Placeholder(Placeholder { id, .. }) => { - let value = param_values.get_placeholders_with_values(id)?; - // Replace the placeholder with the value - Ok(Transformed::yes(Expr::Literal(value))) - } - Expr::ScalarSubquery(qry) => { - let subquery = - Arc::new(qry.subquery.replace_params_with_values(param_values)?); - Ok(Transformed::yes(Expr::ScalarSubquery(Subquery { - subquery, - outer_ref_columns: qry.outer_ref_columns.clone(), - }))) - } - _ => Ok(Transformed::no(expr)), - } - }) - .data() - } - // ------------ // Various implementations for printing out LogicalPlans // ------------ From e161cd65bc910a166ead9d93a17295c25cc08a3c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 12 Apr 2024 07:23:45 -0400 Subject: [PATCH 18/39] fix `NamedStructField should be rewritten in OperatorToFunction` in subquery regression (change `ApplyFunctionRewrites` to use TreeNode API (#10032) * fix NamedStructField should be rewritten in OperatorToFunction in subquery * Use TreeNode rewriter --- .../src/analyzer/function_rewrite.rs | 99 +++++++------------ datafusion/optimizer/src/utils.rs | 44 +++++++++ .../sqllogictest/test_files/subquery.slt | 55 +++++++++++ 3 files changed, 133 insertions(+), 65 deletions(-) diff --git a/datafusion/optimizer/src/analyzer/function_rewrite.rs b/datafusion/optimizer/src/analyzer/function_rewrite.rs index 78f65c5b82ab..deb493e09953 100644 --- a/datafusion/optimizer/src/analyzer/function_rewrite.rs +++ b/datafusion/optimizer/src/analyzer/function_rewrite.rs @@ -19,11 +19,13 @@ use super::AnalyzerRule; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNodeRewriter}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{DFSchema, Result}; -use datafusion_expr::expr_rewriter::{rewrite_preserving_name, FunctionRewrite}; + +use crate::utils::NamePreserver; +use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::utils::merge_schema; -use datafusion_expr::{Expr, LogicalPlan}; +use datafusion_expr::LogicalPlan; use std::sync::Arc; /// Analyzer rule that invokes [`FunctionRewrite`]s on expressions @@ -37,36 +39,18 @@ impl ApplyFunctionRewrites { pub fn new(function_rewrites: Vec>) -> Self { Self { function_rewrites } } -} - -impl AnalyzerRule for ApplyFunctionRewrites { - fn name(&self) -> &str { - "apply_function_rewrites" - } - - fn analyze(&self, plan: LogicalPlan, options: &ConfigOptions) -> Result { - self.analyze_internal(&plan, options) - } -} -impl ApplyFunctionRewrites { - fn analyze_internal( + /// Rewrite a single plan, and all its expressions using the provided rewriters + fn rewrite_plan( &self, - plan: &LogicalPlan, + plan: LogicalPlan, options: &ConfigOptions, - ) -> Result { - // optimize child plans first - let new_inputs = plan - .inputs() - .iter() - .map(|p| self.analyze_internal(p, options)) - .collect::>>()?; - + ) -> Result> { // get schema representing all available input fields. This is used for data type // resolution only, so order does not matter here - let mut schema = merge_schema(new_inputs.iter().collect()); + let mut schema = merge_schema(plan.inputs()); - if let LogicalPlan::TableScan(ts) = plan { + if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( ts.table_name.clone(), &ts.source.schema(), @@ -74,49 +58,34 @@ impl ApplyFunctionRewrites { schema.merge(&source_schema); } - let mut expr_rewrite = OperatorToFunctionRewriter { - function_rewrites: &self.function_rewrites, - options, - schema: &schema, - }; + let name_preserver = NamePreserver::new(&plan); + + plan.map_expressions(|expr| { + let original_name = name_preserver.save(&expr)?; - let new_expr = plan - .expressions() - .into_iter() - .map(|expr| { - // ensure names don't change: - // https://github.com/apache/arrow-datafusion/issues/3555 - rewrite_preserving_name(expr, &mut expr_rewrite) - }) - .collect::>>()?; + // recursively transform the expression, applying the rewrites at each step + let result = expr.transform_up(&|expr| { + let mut result = Transformed::no(expr); + for rewriter in self.function_rewrites.iter() { + result = result.transform_data(|expr| { + rewriter.rewrite(expr, &schema, options) + })?; + } + Ok(result) + })?; - plan.with_new_exprs(new_expr, new_inputs) + result.map_data(|expr| original_name.restore(expr)) + }) } } -struct OperatorToFunctionRewriter<'a> { - function_rewrites: &'a [Arc], - options: &'a ConfigOptions, - schema: &'a DFSchema, -} - -impl<'a> TreeNodeRewriter for OperatorToFunctionRewriter<'a> { - type Node = Expr; - fn f_up(&mut self, mut expr: Expr) -> Result> { - // apply transforms one by one - let mut transformed = false; - for rewriter in self.function_rewrites.iter() { - let result = rewriter.rewrite(expr, self.schema, self.options)?; - if result.transformed { - transformed = true; - } - expr = result.data - } +impl AnalyzerRule for ApplyFunctionRewrites { + fn name(&self) -> &str { + "apply_function_rewrites" + } - Ok(if transformed { - Transformed::yes(expr) - } else { - Transformed::no(expr) - }) + fn analyze(&self, plan: LogicalPlan, options: &ConfigOptions) -> Result { + plan.transform_up_with_subqueries(&|plan| self.rewrite_plan(plan, options)) + .map(|res| res.data) } } diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 560c63b18882..f0605018e6f3 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -288,3 +288,47 @@ pub fn only_or_err(slice: &[T]) -> Result<&T> { pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { expr_utils::merge_schema(inputs) } + +/// Handles ensuring the name of rewritten expressions is not changed. +/// +/// For example, if an expression `1 + 2` is rewritten to `3`, the name of the +/// expression should be preserved: `3 as "1 + 2"` +/// +/// See for details +pub struct NamePreserver { + use_alias: bool, +} + +/// If the name of an expression is remembered, it will be preserved when +/// rewriting the expression +pub struct SavedName(Option); + +impl NamePreserver { + /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan + pub fn new(plan: &LogicalPlan) -> Self { + Self { + use_alias: !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)), + } + } + + pub fn save(&self, expr: &Expr) -> Result { + let original_name = if self.use_alias { + Some(expr.name_for_alias()?) + } else { + None + }; + + Ok(SavedName(original_name)) + } +} + +impl SavedName { + /// Ensures the name of the rewritten expression is preserved + pub fn restore(self, expr: Expr) -> Result { + let Self(original_name) = self; + match original_name { + Some(name) => expr.alias_if_changed(name), + None => Ok(expr), + } + } +} diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index cc6428e51435..1ae89c9159f8 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1060,3 +1060,58 @@ logical_plan Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) --Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a ----TableScan: t projection=[a] + +### +## Ensure that operators are rewritten in subqueries +### + +statement ok +create table foo(x int) as values (1); + +# Show input data +query ? +select struct(1, 'b') +---- +{c0: 1, c1: b} + + +query T +select (select struct(1, 'b')['c1']); +---- +b + +query T +select 'foo' || (select struct(1, 'b')['c1']); +---- +foob + +query I +SELECT * FROM (VALUES (1), (2)) +WHERE column1 IN (SELECT struct(1, 'b')['c0']); +---- +1 + +# also add an expression so the subquery is the output expr +query I +SELECT * FROM (VALUES (1), (2)) +WHERE 1+2 = 3 AND column1 IN (SELECT struct(1, 'b')['c0']); +---- +1 + + +query I +SELECT * FROM foo +WHERE EXISTS (SELECT * FROM (values (1)) WHERE column1 = foo.x AND struct(1, 'b')['c0'] = 1); +---- +1 + +# also add an expression so the subquery is the output expr +query I +SELECT * FROM foo +WHERE 1+2 = 3 AND EXISTS (SELECT * FROM (values (1)) WHERE column1 = foo.x AND struct(1, 'b')['c0'] = 1); +---- +1 + + +statement ok +drop table foo; From a5cf0b8902ae55b81ac86b875c7e94cf1bdc205d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 12 Apr 2024 07:37:06 -0400 Subject: [PATCH 19/39] Avoid copies in `InlineTableScan` via TreeNode API (#10038) * Avoid copies in `InlineTableScan` via TreeNode API * Improve variable name --- .../src/analyzer/inline_table_scan.rs | 93 +++++++------------ 1 file changed, 31 insertions(+), 62 deletions(-) diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index 88202ffd21f1..cc5f870a9c73 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -17,17 +17,13 @@ //! Analyzed rule to replace TableScan references //! such as DataFrames and Views and inlines the LogicalPlan. -use std::sync::Arc; use crate::analyzer::AnalyzerRule; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Column, Result}; -use datafusion_expr::expr::{Exists, InSubquery}; -use datafusion_expr::{ - logical_plan::LogicalPlan, Expr, Filter, LogicalPlanBuilder, TableScan, -}; +use datafusion_expr::{logical_plan::LogicalPlan, Expr, LogicalPlanBuilder, TableScan}; /// Analyzed rule that inlines TableScan that provide a [`LogicalPlan`] /// (DataFrame / ViewTable) @@ -51,65 +47,38 @@ impl AnalyzerRule for InlineTableScan { } fn analyze_internal(plan: LogicalPlan) -> Result> { - match plan { - // Match only on scans without filter / projection / fetch - // Views and DataFrames won't have those added - // during the early stage of planning - LogicalPlan::TableScan(TableScan { - table_name, - source, - projection, - filters, - .. - }) if filters.is_empty() && source.get_logical_plan().is_some() => { - let sub_plan = source.get_logical_plan().unwrap(); - let projection_exprs = generate_projection_expr(&projection, sub_plan)?; - LogicalPlanBuilder::from(sub_plan.clone()) - .project(projection_exprs)? - // Ensures that the reference to the inlined table remains the - // same, meaning we don't have to change any of the parent nodes - // that reference this table. - .alias(table_name)? - .build() - .map(Transformed::yes) - } - LogicalPlan::Filter(filter) => { - let new_expr = filter.predicate.transform(&rewrite_subquery).data()?; - Filter::try_new(new_expr, filter.input) - .map(|e| Transformed::yes(LogicalPlan::Filter(e))) + // rewrite any subqueries in the plan first + let transformed_plan = + plan.map_subqueries(|plan| plan.transform_up(&analyze_internal))?; + + let transformed_plan = transformed_plan.transform_data(|plan| { + match plan { + // Match only on scans without filter / projection / fetch + // Views and DataFrames won't have those added + // during the early stage of planning + LogicalPlan::TableScan(TableScan { + table_name, + source, + projection, + filters, + .. + }) if filters.is_empty() && source.get_logical_plan().is_some() => { + let sub_plan = source.get_logical_plan().unwrap(); + let projection_exprs = generate_projection_expr(&projection, sub_plan)?; + LogicalPlanBuilder::from(sub_plan.clone()) + .project(projection_exprs)? + // Ensures that the reference to the inlined table remains the + // same, meaning we don't have to change any of the parent nodes + // that reference this table. + .alias(table_name)? + .build() + .map(Transformed::yes) + } + _ => Ok(Transformed::no(plan)), } - _ => Ok(Transformed::no(plan)), - } -} + })?; -fn rewrite_subquery(expr: Expr) -> Result> { - match expr { - Expr::Exists(Exists { subquery, negated }) => { - let plan = subquery.subquery.as_ref().clone(); - let new_plan = plan.transform_up(&analyze_internal).data()?; - let subquery = subquery.with_plan(Arc::new(new_plan)); - Ok(Transformed::yes(Expr::Exists(Exists { subquery, negated }))) - } - Expr::InSubquery(InSubquery { - expr, - subquery, - negated, - }) => { - let plan = subquery.subquery.as_ref().clone(); - let new_plan = plan.transform_up(&analyze_internal).data()?; - let subquery = subquery.with_plan(Arc::new(new_plan)); - Ok(Transformed::yes(Expr::InSubquery(InSubquery::new( - expr, subquery, negated, - )))) - } - Expr::ScalarSubquery(subquery) => { - let plan = subquery.subquery.as_ref().clone(); - let new_plan = plan.transform_up(&analyze_internal).data()?; - let subquery = subquery.with_plan(Arc::new(new_plan)); - Ok(Transformed::yes(Expr::ScalarSubquery(subquery))) - } - _ => Ok(Transformed::no(expr)), - } + Ok(transformed_plan) } fn generate_projection_expr( From 952c98ecc4800df620e97b06463ed7daff227fe6 Mon Sep 17 00:00:00 2001 From: Phillip LeBlanc Date: Sat, 13 Apr 2024 00:36:19 +0900 Subject: [PATCH 20/39] Bump sccache-action to v0.0.4 (#10060) --- .github/actions/setup-rust-runtime/action.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/actions/setup-rust-runtime/action.yaml b/.github/actions/setup-rust-runtime/action.yaml index 1d814055ae28..cd18be989031 100644 --- a/.github/actions/setup-rust-runtime/action.yaml +++ b/.github/actions/setup-rust-runtime/action.yaml @@ -21,7 +21,7 @@ runs: using: "composite" steps: - name: Run sccache-cache - uses: mozilla-actions/sccache-action@v0.0.3 + uses: mozilla-actions/sccache-action@v0.0.4 - name: Configure runtime env shell: bash # do not produce debug symbols to keep memory usage down From 60305ed4ce2b4674efb42d06965963b706b1c2b8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 12 Apr 2024 11:43:29 -0700 Subject: [PATCH 21/39] chore: add GitHub workflow to close stale PRs (#10046) * Add action to close stale issues and PRs * add notes to contributors guide * prettier * Update .github/workflows/stale.yml Co-authored-by: Andrew Lamb * do not close stale issues * update docs * prettier --------- Co-authored-by: Andrew Lamb --- .github/workflows/stale.yml | 38 ++++++++++++++++++++++++++ docs/source/contributor-guide/index.md | 5 ++++ 2 files changed, 43 insertions(+) create mode 100644 .github/workflows/stale.yml diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml new file mode 100644 index 000000000000..2312526824a9 --- /dev/null +++ b/.github/workflows/stale.yml @@ -0,0 +1,38 @@ +# 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. + +name: "Close stale PRs" +on: + schedule: + - cron: "30 1 * * *" + +jobs: + close-stale-prs: + runs-on: ubuntu-latest + permissions: + issues: write + pull-requests: write + steps: + - uses: actions/stale@v9 + with: + stale-pr-message: "Thank you for your contribution. Unfortunately, this pull request is stale because it has been open 60 days with no activity. Please remove the stale label or comment or this will be closed in 7 days." + days-before-pr-stale: 60 + days-before-pr-close: 7 + # do not close stale issues + days-before-issue-stale: -1 + days-before-issue-close: -1 + repo-token: ${{ secrets.GITHUB_TOKEN }} diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index eadf4147c57e..9edc3fa2fb00 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -106,6 +106,11 @@ A "major" PR means there is a substantial change in design or a change in the AP The good thing about open code and open development is that any issues in one change can almost always be fixed with a follow on PR. +## Stale PRs + +Pull requests will be marked with a `stale` label after 60 days of inactivity and then closed 7 days after that. +Commenting on the PR will remove the `stale` label. + ## Getting Started This section describes how you can get started at developing DataFusion. From ffc84445015eb69b2d3352304ec09fbc6267f282 Mon Sep 17 00:00:00 2001 From: Matthew Cramerus <8771538+suremarc@users.noreply.github.com> Date: Fri, 12 Apr 2024 16:39:06 -0500 Subject: [PATCH 22/39] feat: eliminate redundant sorts on monotonic expressions (#9813) * initial impl * add comments & fix name * even more comments * add negative test * updated sqllogictest * make the test easier to read * add collapse_monotonic_lex_req into collapse_lex_req * more tests in sqllogictest * another test in sqllogictest * add yet another negative test case --- .../physical-expr/src/equivalence/mod.rs | 48 ++++++++- .../src/equivalence/properties.rs | 82 ++++++++++++++ .../test_files/filter_without_sort_exec.slt | 102 +++++++++++++++++- 3 files changed, 226 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 46909f23616f..fd8123c45b06 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use crate::expressions::Column; +use crate::sort_properties::SortProperties; use crate::{LexRequirement, PhysicalExpr, PhysicalSortRequirement}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; @@ -35,6 +36,10 @@ pub use properties::{join_equivalence_properties, EquivalenceProperties}; /// This function constructs a duplicate-free `LexOrderingReq` by filtering out /// duplicate entries that have same physical expression inside. For example, /// `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a Some(ASC)]`. +/// +/// It will also filter out entries that are ordered if the next entry is; +/// for instance, `vec![floor(a) Some(ASC), a Some(ASC)]` will be collapsed to +/// `vec![a Some(ASC)]`. pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { let mut output = Vec::::new(); for item in input { @@ -42,7 +47,48 @@ pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { output.push(item); } } - output + collapse_monotonic_lex_req(output) +} + +/// This function constructs a normalized [`LexRequirement`] by filtering out entries +/// that are ordered if the next entry is. +/// Used in `collapse_lex_req` +fn collapse_monotonic_lex_req(input: LexRequirement) -> LexRequirement { + input + .iter() + .enumerate() + .filter_map(|(i, item)| { + // If it's the last entry, there is no next entry + if i == input.len() - 1 { + return Some(item); + } + let next_expr = &input[i + 1]; + + // Only handle expressions with exactly one child + // TODO: it should be possible to handle expressions orderings f(a, b, c), a, b, c + // if f is monotonic in all arguments + if !(item.expr.children().len() == 1 + && item.expr.children()[0].eq(&next_expr.expr)) + { + return Some(item); + } + + let opts = match next_expr.options { + None => return Some(item), + Some(opts) => opts, + }; + + if item.options.map(SortProperties::Ordered) + == Some(item.expr.get_ordering(&[SortProperties::Ordered(opts)])) + { + // Remove the redundant sort + return None; + } + + Some(item) + }) + .cloned() + .collect::>() } /// Adds the `offset` value to `Column` indices inside `expr`. This function is diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index c14c88d6c69b..58ef5ec797b8 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -2212,6 +2212,88 @@ mod tests { ); } + Ok(()) + } + #[test] + fn test_eliminate_redundant_monotonic_sorts() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Date32, true), + Field::new("b", DataType::Utf8, true), + Field::new("c", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let base_properties = EquivalenceProperties::new(schema.clone()).with_reorder( + ["a", "b", "c"] + .into_iter() + .map(|c| { + col(c, schema.as_ref()).map(|expr| PhysicalSortExpr { + expr, + options: SortOptions { + descending: false, + nulls_first: true, + }, + }) + }) + .collect::>>()?, + ); + + struct TestCase { + name: &'static str, + constants: Vec>, + equal_conditions: Vec<[Arc; 2]>, + sort_columns: &'static [&'static str], + should_satisfy_ordering: bool, + } + + let col_a = col("a", schema.as_ref())?; + let col_b = col("b", schema.as_ref())?; + let col_c = col("c", schema.as_ref())?; + let cast_c = Arc::new(CastExpr::new(col_c, DataType::Date32, None)); + + let cases = vec![ + TestCase { + name: "(a, b, c) -> (c)", + // b is constant, so it should be removed from the sort order + constants: vec![col_b], + equal_conditions: vec![[cast_c.clone(), col_a.clone()]], + sort_columns: &["c"], + should_satisfy_ordering: true, + }, + TestCase { + name: "not ordered because (b) is not constant", + // b is not constant anymore + constants: vec![], + // a and c are still compatible, but this is irrelevant since the original ordering is (a, b, c) + equal_conditions: vec![[cast_c.clone(), col_a.clone()]], + sort_columns: &["c"], + should_satisfy_ordering: false, + }, + ]; + + for case in cases { + let mut properties = base_properties.clone().add_constants(case.constants); + for [left, right] in &case.equal_conditions { + properties.add_equal_conditions(left, right) + } + + let sort = case + .sort_columns + .iter() + .map(|&name| { + col(name, &schema).map(|col| PhysicalSortExpr { + expr: col, + options: SortOptions::default(), + }) + }) + .collect::>>()?; + + assert_eq!( + properties.ordering_satisfy(&sort), + case.should_satisfy_ordering, + "failed test '{}'", + case.name + ); + } + Ok(()) } } diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index 05e622db8a02..b2cc64e3a7cf 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -18,9 +18,9 @@ # prepare table statement ok CREATE UNBOUNDED EXTERNAL TABLE data ( - "date" VARCHAR, + "date" DATE, "ticker" VARCHAR, - "time" VARCHAR, + "time" TIMESTAMP, ) STORED AS CSV WITH ORDER ("date", "ticker", "time") LOCATION './a.parquet'; @@ -43,19 +43,111 @@ SortPreservingMergeExec: [date@0 ASC NULLS LAST,time@2 ASC NULLS LAST] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +# constant ticker, CAST(time AS DATE) = time, order by time +query TT +explain SELECT * FROM data +WHERE ticker = 'A' AND CAST(time AS DATE) = date +ORDER BY "time" +---- +logical_plan +Sort: data.time ASC NULLS LAST +--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +----TableScan: data projection=[date, ticker, time] +physical_plan +SortPreservingMergeExec: [time@2 ASC NULLS LAST] +--CoalesceBatchesExec: target_batch_size=8192 +----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] + +# same thing but order by date +query TT +explain SELECT * FROM data +WHERE ticker = 'A' AND CAST(time AS DATE) = date +ORDER BY "date" +---- +logical_plan +Sort: data.date ASC NULLS LAST +--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +----TableScan: data projection=[date, ticker, time] +physical_plan +SortPreservingMergeExec: [date@0 ASC NULLS LAST] +--CoalesceBatchesExec: target_batch_size=8192 +----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] + +# same thing but order by ticker +query TT +explain SELECT * FROM data +WHERE ticker = 'A' AND CAST(time AS DATE) = date +ORDER BY "ticker" +---- +logical_plan +Sort: data.ticker ASC NULLS LAST +--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +----TableScan: data projection=[date, ticker, time] +physical_plan +CoalescePartitionsExec +--CoalesceBatchesExec: target_batch_size=8192 +----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] + +# same thing but order by time, date +query TT +explain SELECT * FROM data +WHERE ticker = 'A' AND CAST(time AS DATE) = date +ORDER BY "time", "date"; +---- +logical_plan +Sort: data.time ASC NULLS LAST, data.date ASC NULLS LAST +--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +----TableScan: data projection=[date, ticker, time] +physical_plan +SortPreservingMergeExec: [time@2 ASC NULLS LAST,date@0 ASC NULLS LAST] +--CoalesceBatchesExec: target_batch_size=8192 +----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] + +# CAST(time AS DATE) <> date (should require a sort) +# no physical plan due to sort breaking pipeline +query TT +explain SELECT * FROM data +WHERE ticker = 'A' AND CAST(time AS DATE) <> date +ORDER BY "time" +---- +logical_plan +Sort: data.time ASC NULLS LAST +--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) != data.date +----TableScan: data projection=[date, ticker, time] + +# no relation between time & date +# should also be pipeline breaking +query TT +explain SELECT * FROM data +WHERE ticker = 'A' +ORDER BY "time" +---- +logical_plan +Sort: data.time ASC NULLS LAST +--Filter: data.ticker = Utf8("A") +----TableScan: data projection=[date, ticker, time] + # query query TT explain SELECT * FROM data -WHERE date = 'A' +WHERE date = '2006-01-02' ORDER BY "ticker", "time"; ---- logical_plan Sort: data.ticker ASC NULLS LAST, data.time ASC NULLS LAST ---Filter: data.date = Utf8("A") +--Filter: data.date = Date32("13150") ----TableScan: data projection=[date, ticker, time] physical_plan SortPreservingMergeExec: [ticker@1 ASC NULLS LAST,time@2 ASC NULLS LAST] --CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: date@0 = A +----FilterExec: date@0 = 13150 ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] From 6872800485a3c749ba9c5b874c1f0d5d87fd359e Mon Sep 17 00:00:00 2001 From: Phillip LeBlanc Date: Sat, 13 Apr 2024 09:23:02 +0900 Subject: [PATCH 23/39] Disable `crypto_expressions` feature properly for --no-default-features (#10059) * Turn on functions * Clean up crypto_expressions feature from unused crates * Disable crypto_expressions from datafusion/functions by default * Update Cargo.lock for datafusion-cli --- datafusion-cli/Cargo.lock | 169 ++++++++++++++-------------- datafusion/core/Cargo.toml | 2 +- datafusion/functions/Cargo.toml | 1 - datafusion/optimizer/Cargo.toml | 3 +- datafusion/physical-expr/Cargo.toml | 6 - 5 files changed, 84 insertions(+), 97 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index a38fd7f1fe28..9e192b0be0f1 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -63,9 +63,9 @@ dependencies = [ [[package]] name = "allocator-api2" -version = "0.2.16" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" +checksum = "5c6cb57a04249c6480766f7f7cef5467412af1490f8d1e243141daddada3264f" [[package]] name = "android-tzdata" @@ -363,9 +363,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.6" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a116f46a969224200a0a97f29cfd4c50e7534e4b4826bd23ea2c3c533039c82c" +checksum = "07dbbf24db18d609b1462965249abdf49129ccad073ec257da372adc83259c60" dependencies = [ "bzip2", "flate2", @@ -375,19 +375,19 @@ dependencies = [ "pin-project-lite", "tokio", "xz2", - "zstd 0.13.0", - "zstd-safe 7.0.0", + "zstd 0.13.1", + "zstd-safe 7.1.0", ] [[package]] name = "async-trait" -version = "0.1.79" +version = "0.1.80" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a507401cad91ec6a857ed5513a2073c82a9b9048762b885bb98655b306964681" +checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -820,9 +820,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.15.4" +version = "3.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ff69b9dd49fd426c69a0db9fc04dd934cdb6645ff000864d98f7e2af8830eaa" +checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "byteorder" @@ -869,9 +869,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.90" +version = "1.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cd6604a82acf3039f1144f54b8eb34e91ffba622051189e71b781822d5ee1f5" +checksum = "2678b2e3449475e95b0aa6f9b506a28e61b3dc8996592b983695e8ebb58a8b41" dependencies = [ "jobserver", "libc", @@ -970,12 +970,12 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.1.0" +version = "7.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" +checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" dependencies = [ - "strum 0.25.0", - "strum_macros 0.25.3", + "strum 0.26.2", + "strum_macros 0.26.2", "unicode-width", ] @@ -1092,7 +1092,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ad291aa74992b9b7a7e88c38acbbf6ad7e107f1d90ee8775b7bc1fc3394f485c" dependencies = [ "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -1163,7 +1163,7 @@ dependencies = [ "url", "uuid", "xz2", - "zstd 0.13.0", + "zstd 0.13.1", ] [[package]] @@ -1339,8 +1339,6 @@ dependencies = [ "arrow-schema", "arrow-string", "base64 0.22.0", - "blake2", - "blake3", "chrono", "datafusion-common", "datafusion-execution", @@ -1353,11 +1351,9 @@ dependencies = [ "indexmap 2.2.6", "itertools", "log", - "md-5", "paste", "petgraph", "regex", - "sha2", ] [[package]] @@ -1495,9 +1491,9 @@ checksum = "11157ac094ffbdde99aa67b23417ebdd801842852b500e395a45a9c0aac03e4a" [[package]] name = "encoding_rs" -version = "0.8.33" +version = "0.8.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7268b386296a025e474d5140678f75d6de9493ae55a5d709eeb9dd08149945e1" +checksum = "b45de904aa0b010bce2ab45264d0631681847fa7b6f2eaa7dab7619943bc4f59" dependencies = [ "cfg-if", ] @@ -1679,7 +1675,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -1730,9 +1726,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.12" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "190092ea657667030ac6a35e305e62fc4dd69fd98ac98631e5d3a2b1575a12b5" +checksum = "94b22e06ecb0110981051723910cbf0b5f5e09a2062dd7663334ee79a9d1286c" dependencies = [ "cfg-if", "libc", @@ -1753,9 +1749,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.25" +version = "0.3.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fbd2820c5e49886948654ab546d0688ff24530286bdcf8fca3cefb16d4618eb" +checksum = "81fe527a889e1532da5c525686d96d4c2e74cdd345badf8dfef9f6b39dd5f5e8" dependencies = [ "bytes", "fnv", @@ -1772,9 +1768,9 @@ dependencies = [ [[package]] name = "half" -version = "2.4.0" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5eceaaeec696539ddaf7b333340f1af35a5aa87ae3e4f3ead0532f72affab2e" +checksum = "6dd08c532ae367adf81c312a4580bc67f1d0fe8bc9c460520283f4c0ff277888" dependencies = [ "cfg-if", "crunchy", @@ -2029,9 +2025,9 @@ checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" [[package]] name = "jobserver" -version = "0.1.28" +version = "0.1.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab46a6e9526ddef3ae7f787c06f0f2600639ba80ea3eade3d8e670a2230f51d6" +checksum = "f08474e32172238f2827bd160c67871cdb2801430f65c3979184dc362e3ca118" dependencies = [ "libc", ] @@ -2163,13 +2159,12 @@ dependencies = [ [[package]] name = "libredox" -version = "0.0.1" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85c833ca1e66078851dba29046874e38f08b2c883700aa29a03ddd3b23814ee8" +checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ "bitflags 2.5.0", "libc", - "redox_syscall", ] [[package]] @@ -2196,9 +2191,9 @@ checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" [[package]] name = "lz4_flex" -version = "0.11.2" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "912b45c753ff5f7f5208307e8ace7d2a2e30d024e26d3509f3dce546c044ce15" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" dependencies = [ "twox-hash", ] @@ -2226,9 +2221,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.1" +version = "2.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "523dc4f511e55ab87b694dc30d0f820d60906ef06413f93d4d7a1385599cc149" +checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" [[package]] name = "mimalloc" @@ -2413,7 +2408,7 @@ dependencies = [ "rand", "reqwest", "ring 0.17.8", - "rustls-pemfile 2.1.1", + "rustls-pemfile 2.1.2", "serde", "serde_json", "snafu", @@ -2511,7 +2506,7 @@ dependencies = [ "thrift", "tokio", "twox-hash", - "zstd 0.13.0", + "zstd 0.13.1", ] [[package]] @@ -2600,14 +2595,14 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] name = "pin-project-lite" -version = "0.2.13" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" +checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" [[package]] name = "pin-utils" @@ -2714,9 +2709,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.35" +version = "1.0.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "291ec9ab5efd934aaf503a6466c5d5251535d108ee747472c3977cc5acc868ef" +checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" dependencies = [ "proc-macro2", ] @@ -2772,9 +2767,9 @@ dependencies = [ [[package]] name = "redox_users" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a18479200779601e498ada4e8c1e1f50e3ee19deb0259c25825a98b5603b2cb4" +checksum = "bd283d9651eeda4b2a83a43c1c91b266c40fd76ecd39a50a8c630ae69dc72891" dependencies = [ "getrandom", "libredox", @@ -2996,19 +2991,19 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.1" +version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f48172685e6ff52a556baa527774f61fcaa884f59daf3375c62a3f1cd2549dab" +checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" dependencies = [ - "base64 0.21.7", + "base64 0.22.0", "rustls-pki-types", ] [[package]] name = "rustls-pki-types" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "868e20fada228fefaf6b652e00cc73623d54f8171e7352c18bb281571f2d92da" +checksum = "ecd36cc4259e3e4514335c4a138c6b43171a8d61d8f5c9348f9fc7529416f247" [[package]] name = "rustls-webpki" @@ -3022,9 +3017,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.14" +version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" +checksum = "80af6f9131f277a45a3fba6ce8e2258037bb0477a67e610d3c1fe046ab31de47" [[package]] name = "rustyline" @@ -3091,9 +3086,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.9.2" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05b64fb303737d99b81884b2c63433e9ae28abebe5eb5045dcdd175dc2ecf4de" +checksum = "770452e37cad93e0a50d5abc3990d2bc351c36d0328f86cefec2f2fb206eaef6" dependencies = [ "bitflags 1.3.2", "core-foundation", @@ -3104,9 +3099,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.9.1" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e932934257d3b408ed8f30db49d85ea163bfe74961f017f405b025af298f0c7a" +checksum = "41f3cc463c0ef97e11c3461a9d3787412d30e8e7eb907c79180c4a57bf7c04ef" dependencies = [ "core-foundation-sys", "libc", @@ -3141,7 +3136,7 @@ checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3276,7 +3271,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3322,7 +3317,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3335,7 +3330,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3357,9 +3352,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.55" +version = "2.0.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "002a1b3dbf967edfafc32655d0f377ab0bb7b994aa1d32c8cc7e9b8bf3ebb8f0" +checksum = "44cfb93f38070beee36b3fef7d4f5a16f27751d94b187b666a5cc5e9b0d30687" dependencies = [ "proc-macro2", "quote", @@ -3443,7 +3438,7 @@ checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3459,9 +3454,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.34" +version = "0.3.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8248b6521bb14bc45b4067159b9b6ad792e2d6d754d6c41fb50e29fefe38749" +checksum = "5dfd88e563464686c916c7e46e623e520ddc6d79fa6641390f2e3fa86e83e885" dependencies = [ "deranged", "num-conv", @@ -3479,9 +3474,9 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.17" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ba3a3ef41e6672a2f0f001392bb5dcd3ff0a9992d618ca761a11c3121547774" +checksum = "3f252a68540fde3a3877aeea552b832b40ab9a69e318efd078774a01ddee1ccf" dependencies = [ "num-conv", "time-core", @@ -3513,9 +3508,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.36.0" +version = "1.37.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61285f6515fa018fb2d1e46eb21223fff441ee8db5d0f1435e8ab4f5cdb80931" +checksum = "1adbebffeca75fcfd058afa480fb6c0b81e165a0323f9c9d39c9697e37c46787" dependencies = [ "backtrace", "bytes", @@ -3538,7 +3533,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3635,7 +3630,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3680,7 +3675,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -3834,7 +3829,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", "wasm-bindgen-shared", ] @@ -3868,7 +3863,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4126,7 +4121,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.55", + "syn 2.0.58", ] [[package]] @@ -4146,11 +4141,11 @@ dependencies = [ [[package]] name = "zstd" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bffb3309596d527cfcba7dfc6ed6052f1d39dfbd7c867aa2e865e4a449c10110" +checksum = "2d789b1514203a1120ad2429eae43a7bd32b90976a7bb8a05f7ec02fa88cc23a" dependencies = [ - "zstd-safe 7.0.0", + "zstd-safe 7.1.0", ] [[package]] @@ -4165,18 +4160,18 @@ dependencies = [ [[package]] name = "zstd-safe" -version = "7.0.0" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43747c7422e2924c11144d5229878b98180ef8b06cca4ab5af37afc8a8d8ea3e" +checksum = "1cd99b45c6bc03a018c8b8a86025678c87e55526064e38f9df301989dce7ec0a" dependencies = [ "zstd-sys", ] [[package]] name = "zstd-sys" -version = "2.0.9+zstd.1.5.5" +version = "2.0.10+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" +checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" dependencies = [ "cc", "pkg-config", diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 018b5083e031..3ee6471ca966 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -42,7 +42,7 @@ array_expressions = ["datafusion-functions-array"] avro = ["apache-avro", "num-traits", "datafusion-common/avro"] backtrace = ["datafusion-common/backtrace"] compression = ["xz2", "bzip2", "flate2", "zstd", "async-compression", "tokio-util"] -crypto_expressions = ["datafusion-physical-expr/crypto_expressions", "datafusion-optimizer/crypto_expressions"] +crypto_expressions = ["datafusion-functions/crypto_expressions"] datetime_expressions = ["datafusion-functions/datetime_expressions"] default = [ "array_expressions", diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index 66f8b3010fd2..b3ba80968753 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -41,7 +41,6 @@ default = [ "encoding_expressions", "math_expressions", "regex_expressions", - "crypto_expressions", "string_expressions", "unicode_expressions", ] diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index 1d64a22f1463..2b96d0bc5626 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -33,8 +33,7 @@ name = "datafusion_optimizer" path = "src/lib.rs" [features] -crypto_expressions = ["datafusion-physical-expr/crypto_expressions"] -default = ["crypto_expressions", "regex_expressions"] +default = ["regex_expressions"] regex_expressions = ["datafusion-physical-expr/regex_expressions"] [dependencies] diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 423087d2182b..ba8d237bb276 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -33,9 +33,7 @@ name = "datafusion_physical_expr" path = "src/lib.rs" [features] -crypto_expressions = ["md-5", "sha2", "blake2", "blake3"] default = [ - "crypto_expressions", "regex_expressions", "encoding_expressions", ] @@ -53,8 +51,6 @@ arrow-ord = { workspace = true } arrow-schema = { workspace = true } arrow-string = { workspace = true } base64 = { version = "0.22", optional = true } -blake2 = { version = "^0.10.2", optional = true } -blake3 = { version = "1.0", optional = true } chrono = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } @@ -67,11 +63,9 @@ hex = { version = "0.4", optional = true } indexmap = { workspace = true } itertools = { workspace = true, features = ["use_std"] } log = { workspace = true } -md-5 = { version = "^0.10.0", optional = true } paste = "^1.0" petgraph = "0.6.2" regex = { version = "1.8", optional = true } -sha2 = { version = "^0.10.1", optional = true } [dev-dependencies] arrow = { workspace = true, features = ["test_utils"] } From 637293580db0634a4efbd3f52e4700992ee3080d Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Sat, 13 Apr 2024 03:23:43 +0300 Subject: [PATCH 24/39] Return self in EmptyExec and PlaceholderRowExec with_new_children (#10052) Thus preserving the number of partitions in the plan properties. We are using `EmptyExec` with a specific number of partitions for testing. It took me a while to realise the partitions were reset by plan rewriting. --- datafusion/physical-plan/src/empty.rs | 2 +- datafusion/physical-plan/src/placeholder_row.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 8e8eb4d25e32..6178f7c88c07 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -122,7 +122,7 @@ impl ExecutionPlan for EmptyExec { self: Arc, _: Vec>, ) -> Result> { - Ok(Arc::new(EmptyExec::new(self.schema.clone()))) + Ok(self) } fn execute( diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index c047ff5122fe..d781c41d0447 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -140,7 +140,7 @@ impl ExecutionPlan for PlaceholderRowExec { self: Arc, _: Vec>, ) -> Result> { - Ok(Arc::new(PlaceholderRowExec::new(self.schema.clone()))) + Ok(self) } fn execute( From e4ae613846f2ebf78138cd472a8043e18844e632 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 13 Apr 2024 09:16:31 -0400 Subject: [PATCH 25/39] chore(deps): update sqllogictest requirement from 0.19.0 to 0.20.0 (#10057) Updates the requirements on [sqllogictest](https://github.com/risinglightdb/sqllogictest-rs) to permit the latest version. - [Release notes](https://github.com/risinglightdb/sqllogictest-rs/releases) - [Changelog](https://github.com/risinglightdb/sqllogictest-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/risinglightdb/sqllogictest-rs/commits) --- updated-dependencies: - dependency-name: sqllogictest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/sqllogictest/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index c348f2cddc93..384c5b7153c3 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -48,7 +48,7 @@ object_store = { workspace = true } postgres-protocol = { version = "0.6.4", optional = true } postgres-types = { version = "0.2.4", optional = true } rust_decimal = { version = "1.27.0" } -sqllogictest = "0.19.0" +sqllogictest = "0.20.0" sqlparser = { workspace = true } tempfile = { workspace = true } thiserror = { workspace = true } From 694d4b838439224f5bd51d68ad6254b1b46c06ed Mon Sep 17 00:00:00 2001 From: Phillip LeBlanc Date: Sat, 13 Apr 2024 22:23:03 +0900 Subject: [PATCH 26/39] Rename `FileSinkExec` to `DataSinkExec` (#10065) * Rename `FileSinkExec` to `DataSinkExec` * Add deprecation notice * Fix clippy * Make typedef public --- .../core/src/datasource/file_format/arrow.rs | 4 ++-- .../core/src/datasource/file_format/csv.rs | 4 ++-- .../core/src/datasource/file_format/json.rs | 4 ++-- .../src/datasource/file_format/parquet.rs | 4 ++-- datafusion/core/src/datasource/memory.rs | 4 ++-- datafusion/core/src/datasource/provider.rs | 4 ++-- datafusion/core/src/datasource/stream.rs | 4 ++-- datafusion/physical-plan/src/insert.rs | 21 +++++++++++-------- datafusion/proto/src/physical_plan/mod.rs | 10 ++++----- .../tests/cases/roundtrip_physical_plan.rs | 10 ++++----- datafusion/sqllogictest/test_files/copy.slt | 4 ++-- .../sqllogictest/test_files/explain.slt | 2 +- datafusion/sqllogictest/test_files/insert.slt | 8 +++---- .../test_files/insert_to_external.slt | 8 +++---- 14 files changed, 47 insertions(+), 44 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 99bfbbad9d10..9d58465191e1 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -43,7 +43,7 @@ use arrow_schema::{ArrowError, Schema, SchemaRef}; use datafusion_common::{not_impl_err, DataFusionError, FileType, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; -use datafusion_physical_plan::insert::{DataSink, FileSinkExec}; +use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; @@ -129,7 +129,7 @@ impl FileFormat for ArrowFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(ArrowFileSink::new(conf)); - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, sink, sink_schema, diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index a7849258329b..84235cde0f5d 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -31,7 +31,7 @@ use crate::datasource::physical_plan::{ }; use crate::error::Result; use crate::execution::context::SessionState; -use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::insert::{DataSink, DataSinkExec}; use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; @@ -267,7 +267,7 @@ impl FileFormat for CsvFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(CsvSink::new(conf, writer_options)); - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, sink, sink_schema, diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 0cc38bbb5554..efc0aa4328d8 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -31,7 +31,7 @@ use crate::datasource::physical_plan::FileGroupDisplay; use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec}; use crate::error::Result; use crate::execution::context::SessionState; -use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::insert::{DataSink, DataSinkExec}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, SendableRecordBatchStream, Statistics, }; @@ -177,7 +177,7 @@ impl FileFormat for JsonFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(JsonSink::new(conf, writer_options)); - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, sink, sink_schema, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index bcf4e8a2c8e4..1d41f015121d 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -37,7 +37,7 @@ use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; -use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::insert::{DataSink, DataSinkExec}; use crate::physical_plan::{ Accumulator, DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics, @@ -279,7 +279,7 @@ impl FileFormat for ParquetFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(ParquetSink::new(conf, self.options.clone())); - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, sink, sink_schema, diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 608a46144da3..42e05ebeb33f 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -26,7 +26,7 @@ use crate::datasource::{TableProvider, TableType}; use crate::error::Result; use crate::execution::context::SessionState; use crate::logical_expr::Expr; -use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::insert::{DataSink, DataSinkExec}; use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::{ @@ -279,7 +279,7 @@ impl TableProvider for MemTable { return not_impl_err!("Overwrite not implemented for MemoryTable yet"); } let sink = Arc::new(MemSink::new(self.batches.clone())); - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, sink, self.schema.clone(), diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 100011952b3b..7c58aded3108 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -269,10 +269,10 @@ pub trait TableProvider: Sync + Send { /// /// # See Also /// - /// See [`FileSinkExec`] for the common pattern of inserting a + /// See [`DataSinkExec`] for the common pattern of inserting a /// streams of `RecordBatch`es as files to an ObjectStore. /// - /// [`FileSinkExec`]: crate::physical_plan::insert::FileSinkExec + /// [`DataSinkExec`]: crate::physical_plan::insert::DataSinkExec async fn insert_into( &self, _state: &SessionState, diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 079c1a891d14..2059a5ffcfe4 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -34,7 +34,7 @@ use datafusion_common::{plan_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{CreateExternalTable, Expr, TableType}; -use datafusion_physical_plan::insert::{DataSink, FileSinkExec}; +use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; @@ -277,7 +277,7 @@ impl TableProvider for StreamTable { None => None, }; - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, Arc::new(StreamWrite(self.0.clone())), self.0.schema.clone(), diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index f0233264f280..e3f9f2c76d31 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -74,10 +74,13 @@ pub trait DataSink: DisplayAs + Debug + Send + Sync { ) -> Result; } +#[deprecated(since = "38.0.0", note = "Use [`DataSinkExec`] instead")] +pub type FileSinkExec = DataSinkExec; + /// Execution plan for writing record batches to a [`DataSink`] /// /// Returns a single row with the number of values written -pub struct FileSinkExec { +pub struct DataSinkExec { /// Input plan that produces the record batches to be written. input: Arc, /// Sink to which to write @@ -91,13 +94,13 @@ pub struct FileSinkExec { cache: PlanProperties, } -impl fmt::Debug for FileSinkExec { +impl fmt::Debug for DataSinkExec { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "FileSinkExec schema: {:?}", self.count_schema) + write!(f, "DataSinkExec schema: {:?}", self.count_schema) } } -impl FileSinkExec { +impl DataSinkExec { /// Create a plan to write to `sink` pub fn new( input: Arc, @@ -190,7 +193,7 @@ impl FileSinkExec { } } -impl DisplayAs for FileSinkExec { +impl DisplayAs for DataSinkExec { fn fmt_as( &self, t: DisplayFormatType, @@ -198,16 +201,16 @@ impl DisplayAs for FileSinkExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "FileSinkExec: sink=")?; + write!(f, "DataSinkExec: sink=")?; self.sink.fmt_as(t, f) } } } } -impl ExecutionPlan for FileSinkExec { +impl ExecutionPlan for DataSinkExec { fn name(&self) -> &'static str { - "FileSinkExec" + "DataSinkExec" } /// Return a reference to Any that can be used for downcasting @@ -269,7 +272,7 @@ impl ExecutionPlan for FileSinkExec { context: Arc, ) -> Result { if partition != 0 { - return internal_err!("FileSinkExec can only be called on partition 0!"); + return internal_err!("DataSinkExec can only be called on partition 0!"); } let data = self.execute_input_stream(0, context.clone())?; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 4d5d6cadad17..4d95c847bf99 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -58,7 +58,7 @@ use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::explain::ExplainExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::filter::FilterExec; -use datafusion::physical_plan::insert::FileSinkExec; +use datafusion::physical_plan::insert::DataSinkExec; use datafusion::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion::physical_plan::joins::{ CrossJoinExec, NestedLoopJoinExec, StreamJoinPartitionMode, SymmetricHashJoinExec, @@ -1033,7 +1033,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) }) .transpose()?; - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), Arc::new(sink_schema), @@ -1063,7 +1063,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) }) .transpose()?; - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), Arc::new(sink_schema), @@ -1093,7 +1093,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) }) .transpose()?; - Ok(Arc::new(FileSinkExec::new( + Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), Arc::new(sink_schema), @@ -1879,7 +1879,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }); } - if let Some(exec) = plan.downcast_ref::() { + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index a74b1a38935b..f97cfea765bf 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -48,7 +48,7 @@ use datafusion::physical_plan::expressions::{ NotExpr, NthValue, PhysicalSortExpr, StringAgg, Sum, }; use datafusion::physical_plan::filter::FilterExec; -use datafusion::physical_plan::insert::FileSinkExec; +use datafusion::physical_plan::insert::DataSinkExec; use datafusion::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, }; @@ -861,7 +861,7 @@ fn roundtrip_json_sink() -> Result<()> { }), )]; - roundtrip_test(Arc::new(FileSinkExec::new( + roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, schema.clone(), @@ -896,7 +896,7 @@ fn roundtrip_csv_sink() -> Result<()> { }), )]; - let roundtrip_plan = roundtrip_test_and_return(Arc::new(FileSinkExec::new( + let roundtrip_plan = roundtrip_test_and_return(Arc::new(DataSinkExec::new( input, data_sink, schema.clone(), @@ -906,7 +906,7 @@ fn roundtrip_csv_sink() -> Result<()> { let roundtrip_plan = roundtrip_plan .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); let csv_sink = roundtrip_plan .sink() @@ -948,7 +948,7 @@ fn roundtrip_parquet_sink() -> Result<()> { }), )]; - roundtrip_test(Arc::new(FileSinkExec::new( + roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, schema.clone(), diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index 75f1ccb07aac..0991868cdf5d 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -149,7 +149,7 @@ logical_plan CopyTo: format=parquet output_url=test_files/scratch/copy/table/ options: (format.compression zstd(10)) --TableScan: source_table projection=[col1, col2] physical_plan -FileSinkExec: sink=ParquetSink(file_groups=[]) +DataSinkExec: sink=ParquetSink(file_groups=[]) --MemoryExec: partitions=1, partition_sizes=[1] # Error case @@ -163,7 +163,7 @@ logical_plan CopyTo: format=parquet output_url=test_files/scratch/copy/table/ options: () --TableScan: source_table projection=[col1, col2] physical_plan -FileSinkExec: sink=ParquetSink(file_groups=[]) +DataSinkExec: sink=ParquetSink(file_groups=[]) --MemoryExec: partitions=1, partition_sizes=[1] # Copy more files to directory via query diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index b7ad36dace16..a38d254e051f 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -168,7 +168,7 @@ Dml: op=[Insert Into] table=[sink_table] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan -FileSinkExec: sink=StreamWrite { location: "../../testing/data/csv/aggregate_test_100.csv", batch_size: 8192, encoding: Csv, header: true, .. } +DataSinkExec: sink=StreamWrite { location: "../../testing/data/csv/aggregate_test_100.csv", batch_size: 8192, encoding: Csv, header: true, .. } --SortExec: expr=[c1@0 ASC NULLS LAST] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index b3fbb33e68e7..b16a169598e7 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -64,7 +64,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=MemoryTable (partitions=1) +DataSinkExec: sink=MemoryTable (partitions=1) --ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] @@ -125,7 +125,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=MemoryTable (partitions=1) +DataSinkExec: sink=MemoryTable (partitions=1) --CoalescePartitionsExec ----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] @@ -175,7 +175,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=MemoryTable (partitions=8) +DataSinkExec: sink=MemoryTable (partitions=8) --ProjectionExec: expr=[a1@0 as a1, a2@1 as a2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] @@ -217,7 +217,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1] physical_plan -FileSinkExec: sink=MemoryTable (partitions=1) +DataSinkExec: sink=MemoryTable (partitions=1) --SortExec: expr=[c1@0 ASC NULLS LAST] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 5f100953aff4..0033b070ec1a 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -125,7 +125,7 @@ Dml: op=[Insert Into] table=[ordered_insert_test] --Projection: column1 AS a, column2 AS b ----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... physical_plan -FileSinkExec: sink=CsvSink(file_groups=[]) +DataSinkExec: sink=CsvSink(file_groups=[]) --SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] ----ProjectionExec: expr=[column1@0 as a, column2@1 as b] ------ValuesExec @@ -353,7 +353,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=ParquetSink(file_groups=[]) +DataSinkExec: sink=ParquetSink(file_groups=[]) --ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] @@ -415,7 +415,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=ParquetSink(file_groups=[]) +DataSinkExec: sink=ParquetSink(file_groups=[]) --CoalescePartitionsExec ----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] @@ -458,7 +458,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1] physical_plan -FileSinkExec: sink=ParquetSink(file_groups=[]) +DataSinkExec: sink=ParquetSink(file_groups=[]) --SortExec: expr=[c1@0 ASC NULLS LAST] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true From d698d9deb6d06214ea36cc59739993ddf6441b6a Mon Sep 17 00:00:00 2001 From: advancedxy Date: Sun, 14 Apr 2024 04:05:38 +0800 Subject: [PATCH 27/39] fix: Support Dict types in `in_list` physical plans (#10031) * fix: Relax type check with dict types in in_list * refine comments * fix style, refine comments and address reviewer's comments * refine comments * address comments --- .../physical-expr/src/expressions/in_list.rs | 126 +++++++++++++++++- .../sqllogictest/test_files/dictionary.slt | 39 ++++++ 2 files changed, 161 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index ecdb03e97ee3..07185b4d6527 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -415,6 +415,18 @@ impl PartialEq for InListExpr { } } +/// Checks if two types are logically equal, dictionary types are compared by their value types. +fn is_logically_eq(lhs: &DataType, rhs: &DataType) -> bool { + match (lhs, rhs) { + (DataType::Dictionary(_, v1), DataType::Dictionary(_, v2)) => { + v1.as_ref().eq(v2.as_ref()) + } + (DataType::Dictionary(_, l), _) => l.as_ref().eq(rhs), + (_, DataType::Dictionary(_, r)) => lhs.eq(r.as_ref()), + _ => lhs.eq(rhs), + } +} + /// Creates a unary expression InList pub fn in_list( expr: Arc, @@ -426,7 +438,7 @@ pub fn in_list( let expr_data_type = expr.data_type(schema)?; for list_expr in list.iter() { let list_expr_data_type = list_expr.data_type(schema)?; - if !expr_data_type.eq(&list_expr_data_type) { + if !is_logically_eq(&expr_data_type, &list_expr_data_type) { return internal_err!( "The data type inlist should be same, the value type is {expr_data_type}, one of list expr type is {list_expr_data_type}" ); @@ -499,7 +511,21 @@ mod tests { macro_rules! in_list { ($BATCH:expr, $LIST:expr, $NEGATED:expr, $EXPECTED:expr, $COL:expr, $SCHEMA:expr) => {{ let (cast_expr, cast_list_exprs) = in_list_cast($COL, $LIST, $SCHEMA)?; - let expr = in_list(cast_expr, cast_list_exprs, $NEGATED, $SCHEMA).unwrap(); + in_list_raw!( + $BATCH, + cast_list_exprs, + $NEGATED, + $EXPECTED, + cast_expr, + $SCHEMA + ); + }}; + } + + // applies the in_list expr to an input batch and list without cast + macro_rules! in_list_raw { + ($BATCH:expr, $LIST:expr, $NEGATED:expr, $EXPECTED:expr, $COL:expr, $SCHEMA:expr) => {{ + let expr = in_list($COL, $LIST, $NEGATED, $SCHEMA).unwrap(); let result = expr .evaluate(&$BATCH)? .into_array($BATCH.num_rows()) @@ -540,7 +566,7 @@ mod tests { &schema ); - // expression: "a not in ("a", "b")" + // expression: "a in ("a", "b", null)" let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))]; in_list!( batch, @@ -551,7 +577,7 @@ mod tests { &schema ); - // expression: "a not in ("a", "b")" + // expression: "a not in ("a", "b", null)" let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))]; in_list!( batch, @@ -1314,4 +1340,96 @@ mod tests { Ok(()) } + + #[test] + fn in_list_utf8_with_dict_types() -> Result<()> { + fn dict_lit(key_type: DataType, value: &str) -> Arc { + lit(ScalarValue::Dictionary( + Box::new(key_type), + Box::new(ScalarValue::new_utf8(value.to_string())), + )) + } + + fn null_dict_lit(key_type: DataType) -> Arc { + lit(ScalarValue::Dictionary( + Box::new(key_type), + Box::new(ScalarValue::Utf8(None)), + )) + } + + let schema = Schema::new(vec![Field::new( + "a", + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), + true, + )]); + let a: UInt16DictionaryArray = + vec![Some("a"), Some("d"), None].into_iter().collect(); + let col_a = col("a", &schema)?; + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a)])?; + + // expression: "a in ("a", "b")" + let lists = [ + vec![lit("a"), lit("b")], + vec![ + dict_lit(DataType::Int8, "a"), + dict_lit(DataType::UInt16, "b"), + ], + ]; + for list in lists.iter() { + in_list_raw!( + batch, + list.clone(), + &false, + vec![Some(true), Some(false), None], + col_a.clone(), + &schema + ); + } + + // expression: "a not in ("a", "b")" + for list in lists.iter() { + in_list_raw!( + batch, + list.clone(), + &true, + vec![Some(false), Some(true), None], + col_a.clone(), + &schema + ); + } + + // expression: "a in ("a", "b", null)" + let lists = [ + vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))], + vec![ + dict_lit(DataType::Int8, "a"), + dict_lit(DataType::UInt16, "b"), + null_dict_lit(DataType::UInt16), + ], + ]; + for list in lists.iter() { + in_list_raw!( + batch, + list.clone(), + &false, + vec![Some(true), None, None], + col_a.clone(), + &schema + ); + } + + // expression: "a not in ("a", "b", null)" + for list in lists.iter() { + in_list_raw!( + batch, + list.clone(), + &true, + vec![Some(false), None, None], + col_a.clone(), + &schema + ); + } + + Ok(()) + } } diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index af7bf5cb16e8..891a09fbc177 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -87,6 +87,22 @@ f3 Utf8 YES f4 Float64 YES time Timestamp(Nanosecond, None) YES +# in list with dictionary input +query BBB +SELECT + tag_id in ('1000'), '1000' in (tag_id, null), arrow_cast('999','Dictionary(Int32, Utf8)') in (tag_id, null) +FROM m1 +---- +true true NULL +true true NULL +true true NULL +true true NULL +true true NULL +true true NULL +true true NULL +true true NULL +true true NULL +true true NULL # Table m2 with a tag columns `tag_id` and `type`, a field column `f5`, and `time` statement ok @@ -165,6 +181,29 @@ order by date_bin('30 minutes', time) DESC 3 400 600 500 2023-12-04T00:30:00 3 100 300 200 2023-12-04T00:00:00 +# query with in list +query BBBBBBBB +SELECT + type in ('active', 'passive') + , 'active' in (type) + , 'active' in (type, null) + , arrow_cast('passive','Dictionary(Int8, Utf8)') in (type, null) + , tag_id in ('1000', '2000') + , tag_id in ('999') + , '1000' in (tag_id, null) + , arrow_cast('999','Dictionary(Int16, Utf8)') in (tag_id, null) +FROM m2 +---- +true true true NULL true false true NULL +true true true NULL true false true NULL +true true true NULL true false true NULL +true true true NULL true false true NULL +true true true NULL true false true NULL +true true true NULL true false true NULL +true false NULL true true false true NULL +true false NULL true true false true NULL +true false NULL true true false true NULL +true false NULL true true false true NULL # Reproducer for https://github.com/apache/arrow-datafusion/issues/8738 From 671cef85c550969ab2c86d644968a048cb181c0c Mon Sep 17 00:00:00 2001 From: Yang Jiang Date: Sun, 14 Apr 2024 11:26:49 +0800 Subject: [PATCH 28/39] Prune pages are all null in ParquetExec by row_counts and fix NOT NULL prune (#10051) * Prune pages are all null in ParquetExec by row_counts and fix NOT NULL prune * fix clippy * Update datafusion/core/src/physical_optimizer/pruning.rs Co-authored-by: Andrew Lamb * Update datafusion/core/tests/parquet/page_pruning.rs Co-authored-by: Andrew Lamb * Update datafusion/core/tests/parquet/page_pruning.rs Co-authored-by: Andrew Lamb * Update datafusion/core/tests/parquet/page_pruning.rs Co-authored-by: Andrew Lamb * Update datafusion/core/tests/parquet/page_pruning.rs Co-authored-by: Andrew Lamb * remove allocate vec * better way avoid allocate vec * simply expr --------- Co-authored-by: Andrew Lamb --- .../physical_plan/parquet/page_filter.rs | 16 ++++- .../core/src/physical_optimizer/pruning.rs | 49 ++++++++------ datafusion/core/tests/parquet/mod.rs | 65 ++++++++++++++++--- datafusion/core/tests/parquet/page_pruning.rs | 51 +++++++++++++++ .../core/tests/parquet/row_group_pruning.rs | 2 +- 5 files changed, 153 insertions(+), 30 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index c7706f3458d0..402cc106492e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -314,6 +314,7 @@ fn prune_pages_in_one_row_group( col_page_indexes, col_offset_indexes, target_type: &target_type, + num_rows_in_row_group: group.num_rows(), }; match predicate.prune(&pruning_stats) { @@ -385,6 +386,7 @@ struct PagesPruningStatistics<'a> { // target_type means the logical type in schema: like 'DECIMAL' is the logical type, but the // real physical type in parquet file may be `INT32, INT64, FIXED_LEN_BYTE_ARRAY` target_type: &'a Option, + num_rows_in_row_group: i64, } // Extract the min or max value calling `func` from page idex @@ -548,7 +550,19 @@ impl<'a> PruningStatistics for PagesPruningStatistics<'a> { } fn row_counts(&self, _column: &datafusion_common::Column) -> Option { - None + // see https://github.com/apache/arrow-rs/blob/91f0b1771308609ca27db0fb1d2d49571b3980d8/parquet/src/file/metadata.rs#L979-L982 + + let row_count_per_page = self.col_offset_indexes.windows(2).map(|location| { + Some(location[1].first_row_index - location[0].first_row_index) + }); + + // append the last page row count + let row_count_per_page = row_count_per_page.chain(std::iter::once(Some( + self.num_rows_in_row_group + - self.col_offset_indexes.last().unwrap().first_row_index, + ))); + + Some(Arc::new(Int64Array::from_iter(row_count_per_page))) } fn contained( diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index ebb811408fb3..d8a3814d77e1 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -335,7 +335,7 @@ pub trait PruningStatistics { /// `x < 5` | `CASE WHEN x_null_count = x_row_count THEN false ELSE x_max < 5 END` /// `x = 5 AND y = 10` | `CASE WHEN x_null_count = x_row_count THEN false ELSE x_min <= 5 AND 5 <= x_max END AND CASE WHEN y_null_count = y_row_count THEN false ELSE y_min <= 10 AND 10 <= y_max END` /// `x IS NULL` | `x_null_count > 0` -/// `x IS NOT NULL` | `x_null_count = 0` +/// `x IS NOT NULL` | `x_null_count != row_count` /// `CAST(x as int) = 5` | `CASE WHEN x_null_count = x_row_count THEN false ELSE CAST(x_min as int) <= 5 AND 5 <= CAST(x_max as int) END` /// /// ## Predicate Evaluation @@ -1240,10 +1240,10 @@ fn build_single_column_expr( /// returns a pruning expression in terms of IsNull that will evaluate to true /// if the column may contain null, and false if definitely does not /// contain null. -/// If set `with_not` to true: which means is not null -/// Given an expression reference to `expr`, if `expr` is a column expression, -/// returns a pruning expression in terms of IsNotNull that will evaluate to true -/// if the column not contain any null, and false if definitely contain null. +/// If `with_not` is true, build a pruning expression for `col IS NOT NULL`: `col_count != col_null_count` +/// The pruning expression evaluates to true ONLY if the column definitely CONTAINS +/// at least one NULL value. In this case we can know that `IS NOT NULL` can not be true and +/// thus can prune the row group / value fn build_is_null_column_expr( expr: &Arc, schema: &Schema, @@ -1254,26 +1254,37 @@ fn build_is_null_column_expr( let field = schema.field_with_name(col.name()).ok()?; let null_count_field = &Field::new(field.name(), DataType::UInt64, true); - required_columns - .null_count_column_expr(col, expr, null_count_field) - .map(|null_count_column_expr| { - if with_not { - // IsNotNull(column) => null_count = 0 - Arc::new(phys_expr::BinaryExpr::new( - null_count_column_expr, - Operator::Eq, - Arc::new(phys_expr::Literal::new(ScalarValue::UInt64(Some(0)))), - )) as _ - } else { + if with_not { + if let Ok(row_count_expr) = + required_columns.row_count_column_expr(col, expr, null_count_field) + { + required_columns + .null_count_column_expr(col, expr, null_count_field) + .map(|null_count_column_expr| { + // IsNotNull(column) => null_count != row_count + Arc::new(phys_expr::BinaryExpr::new( + null_count_column_expr, + Operator::NotEq, + row_count_expr, + )) as _ + }) + .ok() + } else { + None + } + } else { + required_columns + .null_count_column_expr(col, expr, null_count_field) + .map(|null_count_column_expr| { // IsNull(column) => null_count > 0 Arc::new(phys_expr::BinaryExpr::new( null_count_column_expr, Operator::Gt, Arc::new(phys_expr::Literal::new(ScalarValue::UInt64(Some(0)))), )) as _ - } - }) - .ok() + }) + .ok() + } } else { None } diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index f36afe1976b1..f90d0e8afb4c 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -28,7 +28,7 @@ use arrow::{ record_batch::RecordBatch, util::pretty::pretty_format_batches, }; -use arrow_array::new_null_array; +use arrow_array::make_array; use chrono::{Datelike, Duration, TimeDelta}; use datafusion::{ datasource::{physical_plan::ParquetExec, provider_as_source, TableProvider}, @@ -77,6 +77,7 @@ enum Scenario { ByteArray, PeriodsInColumnNames, WithNullValues, + WithNullValuesPageLevel, } enum Unit { @@ -632,8 +633,13 @@ fn make_names_batch(name: &str, service_name_values: Vec<&str>) -> RecordBatch { RecordBatch::try_new(schema, vec![Arc::new(name), Arc::new(service_name)]).unwrap() } -/// Return record batch with i8, i16, i32, and i64 sequences with all Null values -fn make_all_null_values() -> RecordBatch { +/// Return record batch with i8, i16, i32, and i64 sequences with Null values +/// here 5 rows in page when using Unit::Page +fn make_int_batches_with_null( + null_values: usize, + no_null_values_start: usize, + no_null_values_end: usize, +) -> RecordBatch { let schema = Arc::new(Schema::new(vec![ Field::new("i8", DataType::Int8, true), Field::new("i16", DataType::Int16, true), @@ -641,13 +647,46 @@ fn make_all_null_values() -> RecordBatch { Field::new("i64", DataType::Int64, true), ])); + let v8: Vec = (no_null_values_start as _..no_null_values_end as _).collect(); + let v16: Vec = (no_null_values_start as _..no_null_values_end as _).collect(); + let v32: Vec = (no_null_values_start as _..no_null_values_end as _).collect(); + let v64: Vec = (no_null_values_start as _..no_null_values_end as _).collect(); + RecordBatch::try_new( schema, vec![ - new_null_array(&DataType::Int8, 5), - new_null_array(&DataType::Int16, 5), - new_null_array(&DataType::Int32, 5), - new_null_array(&DataType::Int64, 5), + make_array( + Int8Array::from_iter( + v8.into_iter() + .map(Some) + .chain(std::iter::repeat(None).take(null_values)), + ) + .to_data(), + ), + make_array( + Int16Array::from_iter( + v16.into_iter() + .map(Some) + .chain(std::iter::repeat(None).take(null_values)), + ) + .to_data(), + ), + make_array( + Int32Array::from_iter( + v32.into_iter() + .map(Some) + .chain(std::iter::repeat(None).take(null_values)), + ) + .to_data(), + ), + make_array( + Int64Array::from_iter( + v64.into_iter() + .map(Some) + .chain(std::iter::repeat(None).take(null_values)), + ) + .to_data(), + ), ], ) .unwrap() @@ -824,9 +863,17 @@ fn create_data_batch(scenario: Scenario) -> Vec { } Scenario::WithNullValues => { vec![ - make_all_null_values(), + make_int_batches_with_null(5, 0, 0), make_int_batches(1, 6), - make_all_null_values(), + make_int_batches_with_null(5, 0, 0), + ] + } + Scenario::WithNullValuesPageLevel => { + vec![ + make_int_batches_with_null(5, 1, 6), + make_int_batches(1, 11), + make_int_batches_with_null(1, 1, 10), + make_int_batches_with_null(5, 1, 6), ] } } diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index da9617f13ee9..1615a1c5766a 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -871,6 +871,57 @@ async fn without_pushdown_filter() { assert!(bytes_scanned_with_filter > bytes_scanned_without_filter); } +#[tokio::test] +// Data layout like this: +// row_group1: page1(1~5), page2(All Null) +// row_group2: page1(1~5), page2(6~10) +// row_group3: page1(1~5), page2(6~9 + Null) +// row_group4: page1(1~5), page2(All Null) +// total 40 rows +async fn test_pages_with_null_values() { + test_prune( + Scenario::WithNullValuesPageLevel, + "SELECT * FROM t where i8 <= 6", + Some(0), + // expect prune pages with all null or pages that have only values greater than 6 + // (row_group1, page2), (row_group4, page2) + Some(10), + 22, + ) + .await; + + test_prune( + Scenario::WithNullValuesPageLevel, + "SELECT * FROM t where \"i16\" is not null", + Some(0), + // expect prune (row_group1, page2) and (row_group4, page2) = 10 rows + Some(10), + 29, + ) + .await; + + test_prune( + Scenario::WithNullValuesPageLevel, + "SELECT * FROM t where \"i32\" is null", + Some(0), + // expect prune (row_group1, page1), (row_group2, page1+2), (row_group3, page1), (row_group3, page1) = 25 rows + Some(25), + 11, + ) + .await; + + test_prune( + Scenario::WithNullValuesPageLevel, + "SELECT * FROM t where \"i64\" > 6", + Some(0), + // expect to prune pages where i is all null, or where always <= 5 + // (row_group1, page1+2), (row_group2, page1), (row_group3, page1) (row_group4, page1+2) = 30 rows + Some(30), + 7, + ) + .await; +} + fn cast_count_metric(metric: MetricValue) -> Option { match metric { MetricValue::Count { count, .. } => Some(count.value()), diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 29bf1ef0a8d4..b3f1fec1753b 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -1296,7 +1296,7 @@ async fn test_row_group_with_null_values() { .test_row_group_prune() .await; - // After pruning, only row group 2should be selected + // After pruning, only row group 2 should be selected RowGroupPruningTest::new() .with_scenario(Scenario::WithNullValues) .with_query("SELECT * FROM t WHERE \"i16\" is Not Null") From 74c7e52377e8fd513356514949f3ce377cbc3e1f Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Mon, 15 Apr 2024 11:52:32 +0200 Subject: [PATCH 29/39] Refactor EliminateOuterJoin to use `rewrite()` (#10081) --- .../optimizer/src/eliminate_outer_join.rs | 71 +++++++++++-------- .../simplify_expressions/simplify_exprs.rs | 2 +- 2 files changed, 42 insertions(+), 31 deletions(-) diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index 63b8b887bb32..c3c5d80922f9 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -17,11 +17,12 @@ //! [`EliminateOuterJoin`] converts `LEFT/RIGHT/FULL` joins to `INNER` joins use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{Column, DFSchema, Result}; +use datafusion_common::{internal_err, Column, DFSchema, Result}; use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan}; -use datafusion_expr::{Expr, Operator}; +use datafusion_expr::{Expr, Filter, Operator}; use crate::optimizer::ApplyOrder; +use datafusion_common::tree_node::Transformed; use datafusion_expr::expr::{BinaryExpr, Cast, TryCast}; use std::sync::Arc; @@ -61,9 +62,29 @@ impl EliminateOuterJoin { impl OptimizerRule for EliminateOuterJoin { fn try_optimize( &self, - plan: &LogicalPlan, + _plan: &LogicalPlan, _config: &dyn OptimizerConfig, ) -> Result> { + internal_err!("Should have called EliminateOuterJoin::rewrite") + } + + fn name(&self) -> &str { + "eliminate_outer_join" + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::TopDown) + } + + fn supports_rewrite(&self) -> bool { + true + } + + fn rewrite( + &self, + plan: LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result> { match plan { LogicalPlan::Filter(filter) => match filter.input.as_ref() { LogicalPlan::Join(join) => { @@ -75,7 +96,7 @@ impl OptimizerRule for EliminateOuterJoin { join.left.schema(), join.right.schema(), true, - )?; + ); let new_join_type = if join.join_type.is_outer() { let mut left_non_nullable = false; @@ -96,7 +117,7 @@ impl OptimizerRule for EliminateOuterJoin { } else { join.join_type }; - let new_join = LogicalPlan::Join(Join { + let new_join = Arc::new(LogicalPlan::Join(Join { left: Arc::new((*join.left).clone()), right: Arc::new((*join.right).clone()), join_type: new_join_type, @@ -105,23 +126,15 @@ impl OptimizerRule for EliminateOuterJoin { filter: join.filter.clone(), schema: join.schema.clone(), null_equals_null: join.null_equals_null, - }); - let exprs = plan.expressions(); - plan.with_new_exprs(exprs, vec![new_join]).map(Some) + })); + Filter::try_new(filter.predicate, new_join) + .map(|f| Transformed::yes(LogicalPlan::Filter(f))) } - _ => Ok(None), + _ => Ok(Transformed::no(LogicalPlan::Filter(filter))), }, - _ => Ok(None), + _ => Ok(Transformed::no(plan)), } } - - fn name(&self) -> &str { - "eliminate_outer_join" - } - - fn apply_order(&self) -> Option { - Some(ApplyOrder::TopDown) - } } pub fn eliminate_outer( @@ -169,11 +182,10 @@ fn extract_non_nullable_columns( left_schema: &Arc, right_schema: &Arc, top_level: bool, -) -> Result<()> { +) { match expr { Expr::Column(col) => { non_nullable_cols.push(col.clone()); - Ok(()) } Expr::BinaryExpr(BinaryExpr { left, op, right }) => match op { // If one of the inputs are null for these operators, the results should be false. @@ -189,7 +201,7 @@ fn extract_non_nullable_columns( left_schema, right_schema, false, - )?; + ); extract_non_nullable_columns( right, non_nullable_cols, @@ -208,15 +220,15 @@ fn extract_non_nullable_columns( left_schema, right_schema, top_level, - )?; + ); extract_non_nullable_columns( right, non_nullable_cols, left_schema, right_schema, top_level, - )?; - return Ok(()); + ); + return; } let mut left_non_nullable_cols: Vec = vec![]; @@ -228,14 +240,14 @@ fn extract_non_nullable_columns( left_schema, right_schema, top_level, - )?; + ); extract_non_nullable_columns( right, &mut right_non_nullable_cols, left_schema, right_schema, top_level, - )?; + ); // for query: select *** from a left join b where b.c1 ... or b.c2 ... // this can be eliminated to inner join. @@ -259,9 +271,8 @@ fn extract_non_nullable_columns( } } } - Ok(()) } - _ => Ok(()), + _ => {} }, Expr::Not(arg) => extract_non_nullable_columns( arg, @@ -272,7 +283,7 @@ fn extract_non_nullable_columns( ), Expr::IsNotNull(arg) => { if !top_level { - return Ok(()); + return; } extract_non_nullable_columns( arg, @@ -290,7 +301,7 @@ fn extract_non_nullable_columns( right_schema, false, ), - _ => Ok(()), + _ => {} } } diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 17312fa6548a..5b0314af20f4 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -53,7 +53,7 @@ impl OptimizerRule for SimplifyExpressions { _plan: &LogicalPlan, _config: &dyn OptimizerConfig, ) -> Result> { - internal_err!("Should have called SimplifyExpressions::try_optimize_owned") + internal_err!("Should have called SimplifyExpressions::rewrite") } fn name(&self) -> &str { From c3f48d477e98a4fe387b386634601048796b632b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 15 Apr 2024 06:13:03 -0400 Subject: [PATCH 30/39] chore(deps): update substrait requirement from 0.29.0 to 0.30.0 (#10084) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.29.0...v0.30.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index d4800eca90f8..37444e8632c7 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -36,7 +36,7 @@ itertools = { workspace = true } object_store = { workspace = true } prost = "0.12" prost-types = "0.12" -substrait = "0.29.0" +substrait = "0.30.0" [dev-dependencies] tokio = { workspace = true } From 483663bc7956382e1c3d69bbb758424c126e4dde Mon Sep 17 00:00:00 2001 From: JasonLi Date: Mon, 15 Apr 2024 18:22:09 +0800 Subject: [PATCH 31/39] feat: optimize `lower` and `upper` functions (#9971) * feat: optimize lower and upper functions * chore: pass cargo check * chore: pass cargo clippy * fix: lower and upper bug * optimize * using iter to find the first nonascii * chore: rename function * refactor: case_conversion_array function * refactor: remove !string_array.is_nullable() from case_conversion_array --- datafusion/functions/Cargo.toml | 11 ++ datafusion/functions/benches/lower.rs | 91 ++++++++++++++ datafusion/functions/benches/upper.rs | 46 ++++++++ datafusion/functions/src/string/common.rs | 137 +++++++++++++--------- datafusion/functions/src/string/lower.rs | 97 ++++++++++++++- datafusion/functions/src/string/upper.rs | 97 ++++++++++++++- 6 files changed, 420 insertions(+), 59 deletions(-) create mode 100644 datafusion/functions/benches/lower.rs create mode 100644 datafusion/functions/benches/upper.rs diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index b3ba80968753..cf15b490b69f 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -83,6 +83,7 @@ unicode-segmentation = { version = "^1.7.1", optional = true } uuid = { version = "1.7", features = ["v4"], optional = true } [dev-dependencies] +arrow = { workspace = true, features = ["test_utils"] } criterion = "0.5" rand = { workspace = true } rstest = { workspace = true } @@ -117,3 +118,13 @@ required-features = ["unicode_expressions"] harness = false name = "ltrim" required-features = ["string_expressions"] + +[[bench]] +harness = false +name = "lower" +required-features = ["string_expressions"] + +[[bench]] +harness = false +name = "upper" +required-features = ["string_expressions"] diff --git a/datafusion/functions/benches/lower.rs b/datafusion/functions/benches/lower.rs new file mode 100644 index 000000000000..fa963f174e46 --- /dev/null +++ b/datafusion/functions/benches/lower.rs @@ -0,0 +1,91 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +extern crate criterion; + +use arrow::array::{ArrayRef, StringArray}; +use arrow::util::bench_util::create_string_array_with_len; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_expr::ColumnarValue; +use datafusion_functions::string; +use std::sync::Arc; + +/// Create an array of args containing a StringArray, where all the values in the +/// StringArray are ASCII. +/// * `size` - the length of the StringArray, and +/// * `str_len` - the length of the strings within the StringArray. +fn create_args1(size: usize, str_len: usize) -> Vec { + let array = Arc::new(create_string_array_with_len::(size, 0.2, str_len)); + vec![ColumnarValue::Array(array)] +} + +/// Create an array of args containing a StringArray, where the first value in the +/// StringArray is non-ASCII. +/// * `size` - the length of the StringArray, and +/// * `str_len` - the length of the strings within the StringArray. +fn create_args2(size: usize) -> Vec { + let mut items = Vec::with_capacity(size); + items.push("农历新年".to_string()); + for i in 1..size { + items.push(format!("DATAFUSION {}", i)); + } + let array = Arc::new(StringArray::from(items)) as ArrayRef; + vec![ColumnarValue::Array(array)] +} + +/// Create an array of args containing a StringArray, where the middle value of the +/// StringArray is non-ASCII. +/// * `size` - the length of the StringArray, and +/// * `str_len` - the length of the strings within the StringArray. +fn create_args3(size: usize) -> Vec { + let mut items = Vec::with_capacity(size); + let half = size / 2; + for i in 0..half { + items.push(format!("DATAFUSION {}", i)); + } + items.push("Ⱦ".to_string()); + for i in half + 1..size { + items.push(format!("DATAFUSION {}", i)); + } + let array = Arc::new(StringArray::from(items)) as ArrayRef; + vec![ColumnarValue::Array(array)] +} + +fn criterion_benchmark(c: &mut Criterion) { + let lower = string::lower(); + for size in [1024, 4096, 8192] { + let args = create_args1(size, 32); + c.bench_function(&format!("lower_all_values_are_ascii: {}", size), |b| { + b.iter(|| black_box(lower.invoke(&args))) + }); + + let args = create_args2(size); + c.bench_function( + &format!("lower_the_first_value_is_nonascii: {}", size), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + + let args = create_args3(size); + c.bench_function( + &format!("lower_the_middle_value_is_nonascii: {}", size), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + } +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions/benches/upper.rs b/datafusion/functions/benches/upper.rs new file mode 100644 index 000000000000..a3e5fbd7a433 --- /dev/null +++ b/datafusion/functions/benches/upper.rs @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +extern crate criterion; + +use arrow::util::bench_util::create_string_array_with_len; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_expr::ColumnarValue; +use datafusion_functions::string; +use std::sync::Arc; + +/// Create an array of args containing a StringArray, where all the values in the +/// StringArray are ASCII. +/// * `size` - the length of the StringArray, and +/// * `str_len` - the length of the strings within the StringArray. +fn create_args(size: usize, str_len: usize) -> Vec { + let array = Arc::new(create_string_array_with_len::(size, 0.2, str_len)); + vec![ColumnarValue::Array(array)] +} + +fn criterion_benchmark(c: &mut Criterion) { + let upper = string::upper(); + for size in [1024, 4096, 8192] { + let args = create_args(size, 32); + c.bench_function("upper_all_values_are_ascii", |b| { + b.iter(|| black_box(upper.invoke(&args))) + }); + } +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 2b554db3979f..97f9e1d93be5 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -19,8 +19,10 @@ use std::fmt::{Display, Formatter}; use std::sync::Arc; use arrow::array::{ - new_null_array, Array, ArrayRef, GenericStringArray, OffsetSizeTrait, + new_null_array, Array, ArrayRef, GenericStringArray, GenericStringBuilder, + OffsetSizeTrait, }; +use arrow::buffer::Buffer; use arrow::datatypes::DataType; use datafusion_common::cast::as_generic_string_array; @@ -112,80 +114,105 @@ pub(crate) fn general_trim( } } -/// applies a unary expression to `args[0]` that is expected to be downcastable to -/// a `GenericStringArray` and returns a `GenericStringArray` (which may have a different offset) -/// # Errors -/// This function errors when: -/// * the number of arguments is not 1 -/// * the first argument is not castable to a `GenericStringArray` -pub(crate) fn unary_string_function<'a, T, O, F, R>( - args: &[&'a dyn Array], - op: F, - name: &str, -) -> Result> -where - R: AsRef, - O: OffsetSizeTrait, - T: OffsetSizeTrait, - F: Fn(&'a str) -> R, -{ - if args.len() != 1 { - return exec_err!( - "{:?} args were supplied but {} takes exactly one argument", - args.len(), - name - ); - } - - let string_array = as_generic_string_array::(args[0])?; +pub(crate) fn to_lower(args: &[ColumnarValue], name: &str) -> Result { + case_conversion(args, |string| string.to_lowercase(), name) +} - // first map is the iterator, second is for the `Option<_>` - Ok(string_array.iter().map(|string| string.map(&op)).collect()) +pub(crate) fn to_upper(args: &[ColumnarValue], name: &str) -> Result { + case_conversion(args, |string| string.to_uppercase(), name) } -pub(crate) fn handle<'a, F, R>( +fn case_conversion<'a, F>( args: &'a [ColumnarValue], op: F, name: &str, ) -> Result where - R: AsRef, - F: Fn(&'a str) -> R, + F: Fn(&'a str) -> String, { match &args[0] { - ColumnarValue::Array(a) => match a.data_type() { - DataType::Utf8 => { - Ok(ColumnarValue::Array(Arc::new(unary_string_function::< - i32, - i32, - _, - _, - >( - &[a.as_ref()], op, name - )?))) - } - DataType::LargeUtf8 => { - Ok(ColumnarValue::Array(Arc::new(unary_string_function::< - i64, - i64, - _, - _, - >( - &[a.as_ref()], op, name - )?))) - } + ColumnarValue::Array(array) => match array.data_type() { + DataType::Utf8 => Ok(ColumnarValue::Array(case_conversion_array::( + array, op, + )?)), + DataType::LargeUtf8 => Ok(ColumnarValue::Array(case_conversion_array::< + i64, + _, + >(array, op)?)), other => exec_err!("Unsupported data type {other:?} for function {name}"), }, ColumnarValue::Scalar(scalar) => match scalar { ScalarValue::Utf8(a) => { - let result = a.as_ref().map(|x| (op)(x).as_ref().to_string()); + let result = a.as_ref().map(|x| op(x)); Ok(ColumnarValue::Scalar(ScalarValue::Utf8(result))) } ScalarValue::LargeUtf8(a) => { - let result = a.as_ref().map(|x| (op)(x).as_ref().to_string()); + let result = a.as_ref().map(|x| op(x)); Ok(ColumnarValue::Scalar(ScalarValue::LargeUtf8(result))) } other => exec_err!("Unsupported data type {other:?} for function {name}"), }, } } + +fn case_conversion_array<'a, O, F>(array: &'a ArrayRef, op: F) -> Result +where + O: OffsetSizeTrait, + F: Fn(&'a str) -> String, +{ + const PRE_ALLOC_BYTES: usize = 8; + + let string_array = as_generic_string_array::(array)?; + let value_data = string_array.value_data(); + + // All values are ASCII. + if value_data.is_ascii() { + return case_conversion_ascii_array::(string_array, op); + } + + // Values contain non-ASCII. + let item_len = string_array.len(); + let capacity = string_array.value_data().len() + PRE_ALLOC_BYTES; + let mut builder = GenericStringBuilder::::with_capacity(item_len, capacity); + + if string_array.null_count() == 0 { + let iter = + (0..item_len).map(|i| Some(op(unsafe { string_array.value_unchecked(i) }))); + builder.extend(iter); + } else { + let iter = string_array.iter().map(|string| string.map(&op)); + builder.extend(iter); + } + Ok(Arc::new(builder.finish())) +} + +/// All values of string_array are ASCII, and when converting case, there is no changes in the byte +/// array length. Therefore, the StringArray can be treated as a complete ASCII string for +/// case conversion, and we can reuse the offsets buffer and the nulls buffer. +fn case_conversion_ascii_array<'a, O, F>( + string_array: &'a GenericStringArray, + op: F, +) -> Result +where + O: OffsetSizeTrait, + F: Fn(&'a str) -> String, +{ + let value_data = string_array.value_data(); + // SAFETY: all items stored in value_data satisfy UTF8. + // ref: impl ByteArrayNativeType for str {...} + let str_values = unsafe { std::str::from_utf8_unchecked(value_data) }; + + // conversion + let converted_values = op(str_values); + assert_eq!(converted_values.len(), str_values.len()); + let bytes = converted_values.into_bytes(); + + // build result + let values = Buffer::from_vec(bytes); + let offsets = string_array.offsets().clone(); + let nulls = string_array.nulls().cloned(); + // SAFETY: offsets and nulls are consistent with the input array. + Ok(Arc::new(unsafe { + GenericStringArray::::new_unchecked(offsets, values, nulls) + })) +} diff --git a/datafusion/functions/src/string/lower.rs b/datafusion/functions/src/string/lower.rs index a1eff7042211..b9b3840252c5 100644 --- a/datafusion/functions/src/string/lower.rs +++ b/datafusion/functions/src/string/lower.rs @@ -23,7 +23,7 @@ use datafusion_common::Result; use datafusion_expr::ColumnarValue; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; -use crate::string::common::handle; +use crate::string::common::to_lower; use crate::utils::utf8_to_str_type; #[derive(Debug)] @@ -62,6 +62,99 @@ impl ScalarUDFImpl for LowerFunc { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - handle(args, |string| string.to_lowercase(), "lower") + to_lower(args, "lower") + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{ArrayRef, StringArray}; + use std::sync::Arc; + + fn to_lower(input: ArrayRef, expected: ArrayRef) -> Result<()> { + let func = LowerFunc::new(); + let args = vec![ColumnarValue::Array(input)]; + let result = match func.invoke(&args)? { + ColumnarValue::Array(result) => result, + _ => unreachable!(), + }; + assert_eq!(&expected, &result); + Ok(()) + } + + #[test] + fn lower_maybe_optimization() -> Result<()> { + let input = Arc::new(StringArray::from(vec![ + Some("农历新年"), + None, + Some("DATAFUSION"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + let expected = Arc::new(StringArray::from(vec![ + Some("农历新年"), + None, + Some("datafusion"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + to_lower(input, expected) + } + + #[test] + fn lower_full_optimization() -> Result<()> { + let input = Arc::new(StringArray::from(vec![ + Some("ARROW"), + None, + Some("DATAFUSION"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + let expected = Arc::new(StringArray::from(vec![ + Some("arrow"), + None, + Some("datafusion"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + to_lower(input, expected) + } + + #[test] + fn lower_partial_optimization() -> Result<()> { + let input = Arc::new(StringArray::from(vec![ + Some("ARROW"), + None, + Some("DATAFUSION"), + Some("@_"), + Some("0123456789"), + Some(""), + Some("\t\n"), + Some("ὈΔΥΣΣΕΎΣ"), + Some("TSCHÜSS"), + Some("Ⱦ"), // ⱦ: length change + Some("农历新年"), + ])) as ArrayRef; + + let expected = Arc::new(StringArray::from(vec![ + Some("arrow"), + None, + Some("datafusion"), + Some("@_"), + Some("0123456789"), + Some(""), + Some("\t\n"), + Some("ὀδυσσεύς"), + Some("tschüss"), + Some("ⱦ"), + Some("农历新年"), + ])) as ArrayRef; + + to_lower(input, expected) } } diff --git a/datafusion/functions/src/string/upper.rs b/datafusion/functions/src/string/upper.rs index c21824d30d53..8f03d7dc6bbc 100644 --- a/datafusion/functions/src/string/upper.rs +++ b/datafusion/functions/src/string/upper.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::string::common::handle; +use crate::string::common::to_upper; use crate::utils::utf8_to_str_type; use arrow::datatypes::DataType; use datafusion_common::Result; @@ -59,6 +59,99 @@ impl ScalarUDFImpl for UpperFunc { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - handle(args, |string| string.to_uppercase(), "upper") + to_upper(args, "upper") + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::{ArrayRef, StringArray}; + use std::sync::Arc; + + fn to_upper(input: ArrayRef, expected: ArrayRef) -> Result<()> { + let func = UpperFunc::new(); + let args = vec![ColumnarValue::Array(input)]; + let result = match func.invoke(&args)? { + ColumnarValue::Array(result) => result, + _ => unreachable!(), + }; + assert_eq!(&expected, &result); + Ok(()) + } + + #[test] + fn upper_maybe_optimization() -> Result<()> { + let input = Arc::new(StringArray::from(vec![ + Some("农历新年"), + None, + Some("datafusion"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + let expected = Arc::new(StringArray::from(vec![ + Some("农历新年"), + None, + Some("DATAFUSION"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + to_upper(input, expected) + } + + #[test] + fn upper_full_optimization() -> Result<()> { + let input = Arc::new(StringArray::from(vec![ + Some("arrow"), + None, + Some("datafusion"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + let expected = Arc::new(StringArray::from(vec![ + Some("ARROW"), + None, + Some("DATAFUSION"), + Some("0123456789"), + Some(""), + ])) as ArrayRef; + + to_upper(input, expected) + } + + #[test] + fn upper_partial_optimization() -> Result<()> { + let input = Arc::new(StringArray::from(vec![ + Some("arrow"), + None, + Some("datafusion"), + Some("@_"), + Some("0123456789"), + Some(""), + Some("\t\n"), + Some("ὀδυσσεύς"), + Some("tschüß"), + Some("ⱦ"), // Ⱦ: length change + Some("农历新年"), + ])) as ArrayRef; + + let expected = Arc::new(StringArray::from(vec![ + Some("ARROW"), + None, + Some("DATAFUSION"), + Some("@_"), + Some("0123456789"), + Some(""), + Some("\t\n"), + Some("ὈΔΥΣΣΕΎΣ"), + Some("TSCHÜSS"), + Some("Ⱦ"), + Some("农历新年"), + ])) as ArrayRef; + + to_upper(input, expected) } } From 03b314c2ee540846638d24496376489d23890ccc Mon Sep 17 00:00:00 2001 From: Duong Cong Toai <35887761+duongcongtoai@users.noreply.github.com> Date: Mon, 15 Apr 2024 12:25:06 +0200 Subject: [PATCH 32/39] Prepend sqllogictest explain result with line number (#10019) * feat: index each explain result with linenumber * chore: new output * rerun plan * new format * regenerate output * chore: revert version hack * generate for avro * regenerate merge changes --- .../engines/datafusion_engine/normalize.rs | 8 +- .../test_files/agg_func_substitute.slt | 66 +- .../sqllogictest/test_files/aggregate.slt | 216 +-- .../test_files/aggregates_topk.slt | 130 +- datafusion/sqllogictest/test_files/avro.slt | 14 +- datafusion/sqllogictest/test_files/copy.slt | 16 +- .../sqllogictest/test_files/csv_files.slt | 10 +- datafusion/sqllogictest/test_files/cte.slt | 130 +- .../sqllogictest/test_files/distinct_on.slt | 26 +- .../sqllogictest/test_files/explain.slt | 128 +- .../test_files/filter_without_sort_exec.slt | 108 +- .../sqllogictest/test_files/group_by.slt | 1204 ++++++------ datafusion/sqllogictest/test_files/insert.slt | 104 +- .../test_files/insert_to_external.slt | 86 +- datafusion/sqllogictest/test_files/join.slt | 30 +- .../join_disable_repartition_joins.slt | 68 +- datafusion/sqllogictest/test_files/joins.slt | 1414 +++++++------- datafusion/sqllogictest/test_files/json.slt | 14 +- datafusion/sqllogictest/test_files/limit.slt | 72 +- datafusion/sqllogictest/test_files/map.slt | 10 +- .../test_files/monotonic_projection_test.slt | 102 +- .../sqllogictest/test_files/options.slt | 28 +- datafusion/sqllogictest/test_files/order.slt | 196 +- .../sqllogictest/test_files/parquet.slt | 28 +- .../sqllogictest/test_files/predicates.slt | 138 +- .../sqllogictest/test_files/references.slt | 12 +- .../sqllogictest/test_files/repartition.slt | 48 +- .../test_files/repartition_scan.slt | 70 +- datafusion/sqllogictest/test_files/scalar.slt | 16 +- datafusion/sqllogictest/test_files/select.slt | 236 +-- .../test_files/sort_merge_join.slt | 24 +- datafusion/sqllogictest/test_files/struct.slt | 8 +- .../sqllogictest/test_files/subquery.slt | 678 +++---- .../sqllogictest/test_files/timestamps.slt | 2 +- datafusion/sqllogictest/test_files/topk.slt | 12 +- .../sqllogictest/test_files/tpch/q1.slt.part | 34 +- .../sqllogictest/test_files/tpch/q10.slt.part | 112 +- .../sqllogictest/test_files/tpch/q11.slt.part | 158 +- .../sqllogictest/test_files/tpch/q12.slt.part | 54 +- .../sqllogictest/test_files/tpch/q13.slt.part | 70 +- .../sqllogictest/test_files/tpch/q14.slt.part | 48 +- .../sqllogictest/test_files/tpch/q15.slt.part | 110 +- .../sqllogictest/test_files/tpch/q16.slt.part | 98 +- .../sqllogictest/test_files/tpch/q17.slt.part | 76 +- .../sqllogictest/test_files/tpch/q18.slt.part | 94 +- .../sqllogictest/test_files/tpch/q19.slt.part | 54 +- .../sqllogictest/test_files/tpch/q2.slt.part | 238 +-- .../sqllogictest/test_files/tpch/q20.slt.part | 140 +- .../sqllogictest/test_files/tpch/q21.slt.part | 172 +- .../sqllogictest/test_files/tpch/q22.slt.part | 90 +- .../sqllogictest/test_files/tpch/q3.slt.part | 96 +- .../sqllogictest/test_files/tpch/q4.slt.part | 66 +- .../sqllogictest/test_files/tpch/q5.slt.part | 152 +- .../sqllogictest/test_files/tpch/q6.slt.part | 26 +- .../sqllogictest/test_files/tpch/q7.slt.part | 158 +- .../sqllogictest/test_files/tpch/q8.slt.part | 210 +-- .../sqllogictest/test_files/tpch/q9.slt.part | 148 +- datafusion/sqllogictest/test_files/union.slt | 382 ++-- datafusion/sqllogictest/test_files/update.slt | 54 +- datafusion/sqllogictest/test_files/window.slt | 1628 ++++++++--------- 60 files changed, 4962 insertions(+), 4958 deletions(-) diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 04e80b77bb9f..e89bc9bb7b90 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -96,14 +96,18 @@ fn expand_row(mut row: Vec) -> impl Iterator> { // form new rows with each additional line let new_lines: Vec<_> = lines .into_iter() - .map(|l| { + .enumerate() + .map(|(idx, l)| { // replace any leading spaces with '-' as // `sqllogictest` ignores whitespace differences // // See https://github.com/apache/arrow-datafusion/issues/6328 let content = l.trim_start(); let new_prefix = "-".repeat(l.len() - content.len()); - vec![format!("{new_prefix}{content}")] + // maintain for each line a number, so + // reviewing explain result changes is easier + let line_num = idx + 1; + vec![format!("{line_num:02}){new_prefix}{content}")] }) .collect(); diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index ff485e97ca0c..811bfa864ffc 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -39,18 +39,18 @@ EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1] as result GROUP BY a; ---- logical_plan -Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result ---Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] -----TableScan: multiple_ordered_table projection=[a, c] +01)Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result +02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] +03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted -----SortExec: expr=[a@0 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +03)----SortExec: expr=[a@0 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT @@ -59,18 +59,18 @@ EXPLAIN SELECT a, NTH_VALUE(c, 1 ORDER BY c) as result GROUP BY a; ---- logical_plan -Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result ---Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] -----TableScan: multiple_ordered_table projection=[a, c] +01)Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result +02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] +03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted -----SortExec: expr=[a@0 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +03)----SortExec: expr=[a@0 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result @@ -78,18 +78,18 @@ EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result GROUP BY a; ---- logical_plan -Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result ---Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(101)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] -----TableScan: multiple_ordered_table projection=[a, c] +01)Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result +02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(101)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] +03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -----SortExec: expr=[a@0 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +03)----SortExec: expr=[a@0 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II SELECT a, ARRAY_AGG(c ORDER BY c)[1] as result diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 966236db2732..30d5c7243f26 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -129,15 +129,15 @@ query TT explain select array_agg(c1 order by c2 desc, c3) from agg_order; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] ---TableScan: agg_order projection=[c1, c2, c3] +01)Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] +02)--TableScan: agg_order projection=[c1, c2, c3] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] -------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +01)AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +04)------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type statement ok @@ -2611,21 +2611,21 @@ query TT EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -------TableScan: aggregate_test_100 projection=[c3] +01)Limit: skip=0, fetch=5 +02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +04)------TableScan: aggregate_test_100 projection=[c3] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] -------------CoalescePartitionsExec ---------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] +07)------------CoalescePartitionsExec +08)--------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; @@ -2640,16 +2640,16 @@ query TT EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; ---- logical_plan -Limit: skip=4, fetch=5 ---Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] -----TableScan: aggregate_test_100 projection=[c2, c3] +01)Limit: skip=4, fetch=5 +02)--Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] +03)----TableScan: aggregate_test_100 projection=[c2, c3] physical_plan -GlobalLimitExec: skip=4, fetch=5 ---AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +01)GlobalLimitExec: skip=4, fetch=5 +02)--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; @@ -2665,26 +2665,26 @@ query TT EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -----Projection: aggregate_test_100.c3 -------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] ---------Filter: aggregate_test_100.c3 >= Int16(10) AND aggregate_test_100.c3 <= Int16(20) -----------TableScan: aggregate_test_100 projection=[c2, c3], partial_filters=[aggregate_test_100.c3 >= Int16(10), aggregate_test_100.c3 <= Int16(20)] +01)Limit: skip=0, fetch=4 +02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +03)----Projection: aggregate_test_100.c3 +04)------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] +05)--------Filter: aggregate_test_100.c3 >= Int16(10) AND aggregate_test_100.c3 <= Int16(20) +06)----------TableScan: aggregate_test_100 projection=[c2, c3], partial_filters=[aggregate_test_100.c3 >= Int16(10), aggregate_test_100.c3 <= Int16(20)] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[4] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[4] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------ProjectionExec: expr=[c3@1 as c3] -------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 -----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +01)GlobalLimitExec: skip=0, fetch=4 +02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[4] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[4] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------ProjectionExec: expr=[c3@1 as c3] +07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +08)--------------CoalescePartitionsExec +09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; @@ -2699,18 +2699,18 @@ query TT EXPLAIN SELECT max(c1), c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5; ---- logical_plan -Projection: MAX(aggregate_test_100.c1), aggregate_test_100.c2, aggregate_test_100.c3 ---Limit: skip=0, fetch=5 -----Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[MAX(aggregate_test_100.c1)]] -------TableScan: aggregate_test_100 projection=[c1, c2, c3] +01)Projection: MAX(aggregate_test_100.c1), aggregate_test_100.c2, aggregate_test_100.c3 +02)--Limit: skip=0, fetch=5 +03)----Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[MAX(aggregate_test_100.c1)]] +04)------TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[MAX(aggregate_test_100.c1)@2 as MAX(aggregate_test_100.c1), c2@0 as c2, c3@1 as c3] ---GlobalLimitExec: skip=0, fetch=5 -----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[MAX(aggregate_test_100.c1)] -------CoalescePartitionsExec ---------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[MAX(aggregate_test_100.c1)] -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +01)ProjectionExec: expr=[MAX(aggregate_test_100.c1)@2 as MAX(aggregate_test_100.c1), c2@0 as c2, c3@1 as c3] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[MAX(aggregate_test_100.c1)] +04)------CoalescePartitionsExec +05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[MAX(aggregate_test_100.c1)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # TODO(msirek): Extend checking in LimitedDistinctAggregation equal groupings to ignore the order of columns # in the group-by column lists, so the limit could be pushed to the lowest AggregateExec in this case @@ -2718,23 +2718,23 @@ query TT EXPLAIN SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; ---- logical_plan -Limit: skip=10, fetch=3 ---Aggregate: groupBy=[[aggregate_test_100.c3, aggregate_test_100.c2]], aggr=[[]] -----Projection: aggregate_test_100.c3, aggregate_test_100.c2 -------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] ---------TableScan: aggregate_test_100 projection=[c2, c3] +01)Limit: skip=10, fetch=3 +02)--Aggregate: groupBy=[[aggregate_test_100.c3, aggregate_test_100.c2]], aggr=[[]] +03)----Projection: aggregate_test_100.c3, aggregate_test_100.c2 +04)------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] +05)--------TableScan: aggregate_test_100 projection=[c2, c3] physical_plan -GlobalLimitExec: skip=10, fetch=3 ---AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] -------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +01)GlobalLimitExec: skip=10, fetch=3 +02)--AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] +07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +08)--------------CoalescePartitionsExec +09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; @@ -2747,16 +2747,16 @@ query TT EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; ---- logical_plan -Limit: skip=0, fetch=3 ---Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] -----TableScan: aggregate_test_100 projection=[c2, c3] +01)Limit: skip=0, fetch=3 +02)--Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] +03)----TableScan: aggregate_test_100 projection=[c2, c3] physical_plan -GlobalLimitExec: skip=0, fetch=3 ---AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[3] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[(NULL as c2, NULL as c3), (c2@0 as c2, NULL as c3), (c2@0 as c2, c3@1 as c3)], aggr=[] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +01)GlobalLimitExec: skip=0, fetch=3 +02)--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[3] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[(NULL as c2, NULL as c3), (c2@0 as c2, NULL as c3), (c2@0 as c2, c3@1 as c3)], aggr=[] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; @@ -2774,21 +2774,21 @@ query TT EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] -------TableScan: aggregate_test_100 projection=[c3] +01)Limit: skip=0, fetch=5 +02)--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +04)------TableScan: aggregate_test_100 projection=[c3] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] -------------CoalescePartitionsExec ---------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] +07)------------CoalescePartitionsExec +08)--------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true statement ok set datafusion.optimizer.enable_distinct_aggregation_soft_limit = true; @@ -3288,21 +3288,21 @@ query TT EXPLAIN SELECT MIN(col0) FROM empty; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[MIN(empty.col0)]] ---TableScan: empty projection=[col0] +01)Aggregate: groupBy=[[]], aggr=[[MIN(empty.col0)]] +02)--TableScan: empty projection=[col0] physical_plan -ProjectionExec: expr=[NULL as MIN(empty.col0)] ---PlaceholderRowExec +01)ProjectionExec: expr=[NULL as MIN(empty.col0)] +02)--PlaceholderRowExec query TT EXPLAIN SELECT MAX(col0) FROM empty; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[MAX(empty.col0)]] ---TableScan: empty projection=[col0] +01)Aggregate: groupBy=[[]], aggr=[[MAX(empty.col0)]] +02)--TableScan: empty projection=[col0] physical_plan -ProjectionExec: expr=[NULL as MAX(empty.col0)] ---PlaceholderRowExec +01)ProjectionExec: expr=[NULL as MAX(empty.col0)] +02)--PlaceholderRowExec statement ok DROP TABLE empty; diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 3f139ede8c77..ab6d9af7bb81 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -40,20 +40,20 @@ query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: MAX(traces.timestamp) DESC NULLS FIRST, fetch=4 -----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] -------TableScan: traces projection=[trace_id, timestamp] +01)Limit: skip=0, fetch=4 +02)--Sort: MAX(traces.timestamp) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 -----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] -------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TI @@ -95,77 +95,77 @@ query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: MAX(traces.timestamp) DESC NULLS FIRST, fetch=4 -----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] -------TableScan: traces projection=[trace_id, timestamp] +01)Limit: skip=0, fetch=4 +02)--Sort: MAX(traces.timestamp) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 -----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] -------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) desc limit 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: MIN(traces.timestamp) DESC NULLS FIRST, fetch=4 -----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MIN(traces.timestamp)]] -------TableScan: traces projection=[trace_id, timestamp] +01)Limit: skip=0, fetch=4 +02)--Sort: MIN(traces.timestamp) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MIN(traces.timestamp)]] +04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [MIN(traces.timestamp)@1 DESC], fetch=4 -----SortExec: TopK(fetch=4), expr=[MIN(traces.timestamp)@1 DESC] -------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [MIN(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[MIN(traces.timestamp)@1 DESC] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) asc limit 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: MAX(traces.timestamp) ASC NULLS LAST, fetch=4 -----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] -------TableScan: traces projection=[trace_id, timestamp] +01)Limit: skip=0, fetch=4 +02)--Sort: MAX(traces.timestamp) ASC NULLS LAST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [MAX(traces.timestamp)@1 ASC NULLS LAST], fetch=4 -----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [MAX(traces.timestamp)@1 ASC NULLS LAST], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 ASC NULLS LAST] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: traces.trace_id ASC NULLS LAST, fetch=4 -----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] -------TableScan: traces projection=[trace_id, timestamp] +01)Limit: skip=0, fetch=4 +02)--Sort: traces.trace_id ASC NULLS LAST, fetch=4 +03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[MAX(traces.timestamp)]] +04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 -----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TI select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index 92ef2a362815..7b9fbe556fee 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -252,11 +252,11 @@ query TT EXPLAIN SELECT count(*) from alltypes_plain ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---TableScan: alltypes_plain projection=[] +01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--TableScan: alltypes_plain projection=[] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]} +01)AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]} diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index 0991868cdf5d..347304520378 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -146,11 +146,11 @@ query TT EXPLAIN COPY source_table TO 'test_files/scratch/copy/table/' STORED AS PARQUET OPTIONS ('format.compression' 'zstd(10)'); ---- logical_plan -CopyTo: format=parquet output_url=test_files/scratch/copy/table/ options: (format.compression zstd(10)) ---TableScan: source_table projection=[col1, col2] +01)CopyTo: format=parquet output_url=test_files/scratch/copy/table/ options: (format.compression zstd(10)) +02)--TableScan: source_table projection=[col1, col2] physical_plan -DataSinkExec: sink=ParquetSink(file_groups=[]) ---MemoryExec: partitions=1, partition_sizes=[1] +01)DataSinkExec: sink=ParquetSink(file_groups=[]) +02)--MemoryExec: partitions=1, partition_sizes=[1] # Error case query error DataFusion error: Invalid or Unsupported Configuration: Format not explicitly set and unable to get file extension! Use STORED AS to define file format. @@ -160,11 +160,11 @@ query TT EXPLAIN COPY source_table to 'test_files/scratch/copy/table/' STORED AS PARQUET ---- logical_plan -CopyTo: format=parquet output_url=test_files/scratch/copy/table/ options: () ---TableScan: source_table projection=[col1, col2] +01)CopyTo: format=parquet output_url=test_files/scratch/copy/table/ options: () +02)--TableScan: source_table projection=[col1, col2] physical_plan -DataSinkExec: sink=ParquetSink(file_groups=[]) ---MemoryExec: partitions=1, partition_sizes=[1] +01)DataSinkExec: sink=ParquetSink(file_groups=[]) +02)--MemoryExec: partitions=1, partition_sizes=[1] # Copy more files to directory via query query IT diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index ab6847afb6a5..f595cbe7f3b1 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -155,9 +155,9 @@ query TT EXPLAIN SELECT * FROM partitioned_table ORDER BY int_col; ---- logical_plan -Sort: partitioned_table.int_col ASC NULLS LAST ---TableScan: partitioned_table projection=[int_col, string_col, bigint_col, partition_col] +01)Sort: partitioned_table.int_col ASC NULLS LAST +02)--TableScan: partitioned_table projection=[int_col, string_col, bigint_col, partition_col] physical_plan -SortPreservingMergeExec: [int_col@0 ASC NULLS LAST] ---SortExec: expr=[int_col@0 ASC NULLS LAST] -----CsvExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/2.csv]]}, projection=[int_col, string_col, bigint_col, partition_col], has_header=true +01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[int_col@0 ASC NULLS LAST] +03)----CsvExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/2.csv]]}, projection=[int_col, string_col, bigint_col, partition_col], has_header=true diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 24797eae0f85..8b972bc79b03 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -31,13 +31,13 @@ query TT EXPLAIN WITH "NUMBERS" AS (SELECT 1 as a, 2 as b, 3 as c) SELECT "NUMBERS".* FROM "NUMBERS" ---- logical_plan -Projection: NUMBERS.a, NUMBERS.b, NUMBERS.c ---SubqueryAlias: NUMBERS -----Projection: Int64(1) AS a, Int64(2) AS b, Int64(3) AS c -------EmptyRelation +01)Projection: NUMBERS.a, NUMBERS.b, NUMBERS.c +02)--SubqueryAlias: NUMBERS +03)----Projection: Int64(1) AS a, Int64(2) AS b, Int64(3) AS c +04)------EmptyRelation physical_plan -ProjectionExec: expr=[1 as a, 2 as b, 3 as c] ---PlaceholderRowExec +01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] +02)--PlaceholderRowExec # cte_use_same_name_multiple_times statement error DataFusion error: Error during planning: WITH query name "a" specified more than once @@ -105,24 +105,24 @@ EXPLAIN WITH RECURSIVE nodes AS ( SELECT * FROM nodes ---- logical_plan -Projection: nodes.id ---SubqueryAlias: nodes -----RecursiveQuery: is_distinct=false -------Projection: Int64(1) AS id ---------EmptyRelation -------Projection: nodes.id + Int64(1) AS id ---------Filter: nodes.id < Int64(10) -----------TableScan: nodes +01)Projection: nodes.id +02)--SubqueryAlias: nodes +03)----RecursiveQuery: is_distinct=false +04)------Projection: Int64(1) AS id +05)--------EmptyRelation +06)------Projection: nodes.id + Int64(1) AS id +07)--------Filter: nodes.id < Int64(10) +08)----------TableScan: nodes physical_plan -RecursiveQueryExec: name=nodes, is_distinct=false ---ProjectionExec: expr=[1 as id] -----PlaceholderRowExec ---CoalescePartitionsExec -----ProjectionExec: expr=[id@0 + 1 as id] -------CoalesceBatchesExec: target_batch_size=8192 ---------FilterExec: id@0 < 10 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------WorkTableExec: name=nodes +01)RecursiveQueryExec: name=nodes, is_distinct=false +02)--ProjectionExec: expr=[1 as id] +03)----PlaceholderRowExec +04)--CoalescePartitionsExec +05)----ProjectionExec: expr=[id@0 + 1 as id] +06)------CoalesceBatchesExec: target_batch_size=8192 +07)--------FilterExec: id@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=nodes # setup statement ok @@ -151,25 +151,25 @@ SELECT * FROM balances ORDER BY time, name, account_balance ---- logical_plan -Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.account_balance ASC NULLS LAST ---Projection: balances.time, balances.name, balances.account_balance -----SubqueryAlias: balances -------RecursiveQuery: is_distinct=false ---------Projection: balance.time, balance.name, balance.account_balance -----------TableScan: balance ---------Projection: balances.time + Int64(1) AS time, balances.name, balances.account_balance + Int64(10) AS account_balance -----------Filter: balances.time < Int64(10) -------------TableScan: balances +01)Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.account_balance ASC NULLS LAST +02)--Projection: balances.time, balances.name, balances.account_balance +03)----SubqueryAlias: balances +04)------RecursiveQuery: is_distinct=false +05)--------Projection: balance.time, balance.name, balance.account_balance +06)----------TableScan: balance +07)--------Projection: balances.time + Int64(1) AS time, balances.name, balances.account_balance + Int64(10) AS account_balance +08)----------Filter: balances.time < Int64(10) +09)------------TableScan: balances physical_plan -SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST] ---RecursiveQueryExec: name=balances, is_distinct=false -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true -----CoalescePartitionsExec -------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] ---------CoalesceBatchesExec: target_batch_size=2 -----------FilterExec: time@0 < 10 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------WorkTableExec: name=balances +01)SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST] +02)--RecursiveQueryExec: name=balances, is_distinct=false +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true +04)----CoalescePartitionsExec +05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] +06)--------CoalesceBatchesExec: target_batch_size=2 +07)----------FilterExec: time@0 < 10 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------------WorkTableExec: name=balances # recursive CTE with static term derived from table works # note that this is run with batch size set to 2. This should produce multiple batches per iteration since the input @@ -720,31 +720,31 @@ explain WITH RECURSIVE recursive_cte AS ( SELECT * FROM recursive_cte; ---- logical_plan -Projection: recursive_cte.val ---SubqueryAlias: recursive_cte -----RecursiveQuery: is_distinct=false -------Projection: Int64(1) AS val ---------EmptyRelation -------Projection: Int64(2) AS val ---------CrossJoin: -----------Filter: recursive_cte.val < Int64(2) -------------TableScan: recursive_cte -----------SubqueryAlias: sub_cte -------------Projection: Int64(2) AS val ---------------EmptyRelation +01)Projection: recursive_cte.val +02)--SubqueryAlias: recursive_cte +03)----RecursiveQuery: is_distinct=false +04)------Projection: Int64(1) AS val +05)--------EmptyRelation +06)------Projection: Int64(2) AS val +07)--------CrossJoin: +08)----------Filter: recursive_cte.val < Int64(2) +09)------------TableScan: recursive_cte +10)----------SubqueryAlias: sub_cte +11)------------Projection: Int64(2) AS val +12)--------------EmptyRelation physical_plan -RecursiveQueryExec: name=recursive_cte, is_distinct=false ---ProjectionExec: expr=[1 as val] -----PlaceholderRowExec ---ProjectionExec: expr=[2 as val] -----CrossJoinExec -------CoalescePartitionsExec ---------CoalesceBatchesExec: target_batch_size=8182 -----------FilterExec: val@0 < 2 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------WorkTableExec: name=recursive_cte -------ProjectionExec: expr=[2 as val] ---------PlaceholderRowExec +01)RecursiveQueryExec: name=recursive_cte, is_distinct=false +02)--ProjectionExec: expr=[1 as val] +03)----PlaceholderRowExec +04)--ProjectionExec: expr=[2 as val] +05)----CrossJoinExec +06)------CoalescePartitionsExec +07)--------CoalesceBatchesExec: target_batch_size=8182 +08)----------FilterExec: val@0 < 2 +09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------------WorkTableExec: name=recursive_cte +11)------ProjectionExec: expr=[2 as val] +12)--------PlaceholderRowExec # Test issue: https://github.com/apache/arrow-datafusion/issues/9794 # Non-recursive term and recursive term have different types diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 3f609e254839..fc0df1d5f65b 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -89,20 +89,20 @@ query TT EXPLAIN SELECT DISTINCT ON (c1) c3, c2 FROM aggregate_test_100 ORDER BY c1, c3; ---- logical_plan -Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST] AS c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST] AS c2 ---Sort: aggregate_test_100.c1 ASC NULLS LAST -----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] -------TableScan: aggregate_test_100 projection=[c1, c2, c3] +01)Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST] AS c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST] AS c2 +02)--Sort: aggregate_test_100.c1 ASC NULLS LAST +03)----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] +04)------TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] ---SortPreservingMergeExec: [c1@0 ASC NULLS LAST] -----SortExec: expr=[c1@0 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] +02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +03)----SortExec: expr=[c1@0 ASC NULLS LAST] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # ON expressions are not a sub-set of the ORDER BY expressions query error SELECT DISTINCT ON expressions must match initial ORDER BY expressions diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index a38d254e051f..c357391e70b5 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -39,15 +39,15 @@ query TT explain SELECT c1 FROM aggregate_test_100 where c2 > 10 ---- logical_plan -Projection: aggregate_test_100.c1 ---Filter: aggregate_test_100.c2 > Int8(10) -----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] +01)Projection: aggregate_test_100.c1 +02)--Filter: aggregate_test_100.c2 > Int8(10) +03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -ProjectionExec: expr=[c1@0 as c1] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: c2@1 > 10 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true +01)ProjectionExec: expr=[c1@0 as c1] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: c2@1 > 10 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true # explain_csv_exec_scan_config @@ -76,12 +76,12 @@ query TT explain SELECT c1 FROM aggregate_test_100_with_order order by c1 ASC limit 10 ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: aggregate_test_100_with_order.c1 ASC NULLS LAST, fetch=10 -----TableScan: aggregate_test_100_with_order projection=[c1] +01)Limit: skip=0, fetch=10 +02)--Sort: aggregate_test_100_with_order.c1 ASC NULLS LAST, fetch=10 +03)----TableScan: aggregate_test_100_with_order projection=[c1] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=10 +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], has_header=true ## explain_physical_plan_only @@ -93,8 +93,8 @@ query TT EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3) ---- physical_plan -ProjectionExec: expr=[2 as COUNT(*)] ---PlaceholderRowExec +01)ProjectionExec: expr=[2 as COUNT(*)] +02)--PlaceholderRowExec statement ok set datafusion.explain.physical_plan_only = false @@ -163,22 +163,22 @@ query TT EXPLAIN INSERT INTO sink_table SELECT * FROM aggregate_test_100 ORDER by c1 ---- logical_plan -Dml: op=[Insert Into] table=[sink_table] ---Projection: aggregate_test_100.c1 AS c1, aggregate_test_100.c2 AS c2, aggregate_test_100.c3 AS c3, aggregate_test_100.c4 AS c4, aggregate_test_100.c5 AS c5, aggregate_test_100.c6 AS c6, aggregate_test_100.c7 AS c7, aggregate_test_100.c8 AS c8, aggregate_test_100.c9 AS c9, aggregate_test_100.c10 AS c10, aggregate_test_100.c11 AS c11, aggregate_test_100.c12 AS c12, aggregate_test_100.c13 AS c13 -----Sort: aggregate_test_100.c1 ASC NULLS LAST -------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] +01)Dml: op=[Insert Into] table=[sink_table] +02)--Projection: aggregate_test_100.c1 AS c1, aggregate_test_100.c2 AS c2, aggregate_test_100.c3 AS c3, aggregate_test_100.c4 AS c4, aggregate_test_100.c5 AS c5, aggregate_test_100.c6 AS c6, aggregate_test_100.c7 AS c7, aggregate_test_100.c8 AS c8, aggregate_test_100.c9 AS c9, aggregate_test_100.c10 AS c10, aggregate_test_100.c11 AS c11, aggregate_test_100.c12 AS c12, aggregate_test_100.c13 AS c13 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST +04)------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan -DataSinkExec: sink=StreamWrite { location: "../../testing/data/csv/aggregate_test_100.csv", batch_size: 8192, encoding: Csv, header: true, .. } ---SortExec: expr=[c1@0 ASC NULLS LAST] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +01)DataSinkExec: sink=StreamWrite { location: "../../testing/data/csv/aggregate_test_100.csv", batch_size: 8192, encoding: Csv, header: true, .. } +02)--SortExec: expr=[c1@0 ASC NULLS LAST] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true # test EXPLAIN VERBOSE query TT EXPLAIN VERBOSE SELECT a, b, c FROM simple_explain_test ---- initial_logical_plan -Projection: simple_explain_test.a, simple_explain_test.b, simple_explain_test.c ---TableScan: simple_explain_test +01)Projection: simple_explain_test.a, simple_explain_test.b, simple_explain_test.c +02)--TableScan: simple_explain_test logical_plan after apply_function_rewrites SAME TEXT AS ABOVE logical_plan after inline_table_scan SAME TEXT AS ABOVE logical_plan after type_coercion SAME TEXT AS ABOVE @@ -240,8 +240,8 @@ logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] physical_plan after OutputRequirements -OutputRequirementExec ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +01)OutputRequirementExec +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -270,8 +270,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Inexact(10), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Inexact(10), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] # Parquet scan with statistics collected statement ok @@ -284,20 +284,20 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] # explain verbose with both collect & show statistics on query TT EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after OutputRequirements -OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +03)----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -307,14 +307,14 @@ physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] statement ok @@ -325,15 +325,15 @@ query TT EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan -GlobalLimitExec: skip=0, fetch=10 ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +01)GlobalLimitExec: skip=0, fetch=10 +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 initial_physical_plan_with_stats -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after OutputRequirements -OutputRequirementExec ---GlobalLimitExec: skip=0, fetch=10 -----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +01)OutputRequirementExec +02)--GlobalLimitExec: skip=0, fetch=10 +03)----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -343,17 +343,17 @@ physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements -GlobalLimitExec: skip=0, fetch=10 ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +01)GlobalLimitExec: skip=0, fetch=10 +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan -GlobalLimitExec: skip=0, fetch=10 ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +01)GlobalLimitExec: skip=0, fetch=10 +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan_with_stats -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] statement ok @@ -368,21 +368,21 @@ query TT explain select make_array(make_array(1, 2, 3), make_array(4, 5, 6)); ---- logical_plan -Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6))) ---EmptyRelation +01)Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6))) +02)--EmptyRelation physical_plan -ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] ---PlaceholderRowExec +01)ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] +02)--PlaceholderRowExec query TT explain select [[1, 2, 3], [4, 5, 6]]; ---- logical_plan -Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6))) ---EmptyRelation +01)Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6))) +02)--EmptyRelation physical_plan -ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] ---PlaceholderRowExec +01)ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] +02)--PlaceholderRowExec # Explain Struct @@ -390,8 +390,8 @@ query TT explain select struct(1, 2.3, 'abc'); ---- logical_plan -Projection: Struct({c0:1,c1:2.3,c2:abc}) AS struct(Int64(1),Float64(2.3),Utf8("abc")) ---EmptyRelation +01)Projection: Struct({c0:1,c1:2.3,c2:abc}) AS struct(Int64(1),Float64(2.3),Utf8("abc")) +02)--EmptyRelation physical_plan -ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] ---PlaceholderRowExec +01)ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] +02)--PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index b2cc64e3a7cf..d5d3d87b5747 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -33,15 +33,15 @@ WHERE ticker = 'A' ORDER BY "date", "time"; ---- logical_plan -Sort: data.date ASC NULLS LAST, data.time ASC NULLS LAST ---Filter: data.ticker = Utf8("A") -----TableScan: data projection=[date, ticker, time] +01)Sort: data.date ASC NULLS LAST, data.time ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") +03)----TableScan: data projection=[date, ticker, time] physical_plan -SortPreservingMergeExec: [date@0 ASC NULLS LAST,time@2 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: ticker@1 = A -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [date@0 ASC NULLS LAST,time@2 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: ticker@1 = A +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # constant ticker, CAST(time AS DATE) = time, order by time query TT @@ -50,15 +50,15 @@ WHERE ticker = 'A' AND CAST(time AS DATE) = date ORDER BY "time" ---- logical_plan -Sort: data.time ASC NULLS LAST ---Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date -----TableScan: data projection=[date, ticker, time] +01)Sort: data.time ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +03)----TableScan: data projection=[date, ticker, time] physical_plan -SortPreservingMergeExec: [time@2 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by date query TT @@ -67,15 +67,15 @@ WHERE ticker = 'A' AND CAST(time AS DATE) = date ORDER BY "date" ---- logical_plan -Sort: data.date ASC NULLS LAST ---Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date -----TableScan: data projection=[date, ticker, time] +01)Sort: data.date ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +03)----TableScan: data projection=[date, ticker, time] physical_plan -SortPreservingMergeExec: [date@0 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by ticker query TT @@ -84,15 +84,15 @@ WHERE ticker = 'A' AND CAST(time AS DATE) = date ORDER BY "ticker" ---- logical_plan -Sort: data.ticker ASC NULLS LAST ---Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date -----TableScan: data projection=[date, ticker, time] +01)Sort: data.ticker ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +03)----TableScan: data projection=[date, ticker, time] physical_plan -CoalescePartitionsExec ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +01)CoalescePartitionsExec +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by time, date query TT @@ -101,15 +101,15 @@ WHERE ticker = 'A' AND CAST(time AS DATE) = date ORDER BY "time", "date"; ---- logical_plan -Sort: data.time ASC NULLS LAST, data.date ASC NULLS LAST ---Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date -----TableScan: data projection=[date, ticker, time] +01)Sort: data.time ASC NULLS LAST, data.date ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date +03)----TableScan: data projection=[date, ticker, time] physical_plan -SortPreservingMergeExec: [time@2 ASC NULLS LAST,date@0 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [time@2 ASC NULLS LAST,date@0 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # CAST(time AS DATE) <> date (should require a sort) # no physical plan due to sort breaking pipeline @@ -119,9 +119,9 @@ WHERE ticker = 'A' AND CAST(time AS DATE) <> date ORDER BY "time" ---- logical_plan -Sort: data.time ASC NULLS LAST ---Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) != data.date -----TableScan: data projection=[date, ticker, time] +01)Sort: data.time ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) != data.date +03)----TableScan: data projection=[date, ticker, time] # no relation between time & date # should also be pipeline breaking @@ -131,9 +131,9 @@ WHERE ticker = 'A' ORDER BY "time" ---- logical_plan -Sort: data.time ASC NULLS LAST ---Filter: data.ticker = Utf8("A") -----TableScan: data projection=[date, ticker, time] +01)Sort: data.time ASC NULLS LAST +02)--Filter: data.ticker = Utf8("A") +03)----TableScan: data projection=[date, ticker, time] # query query TT @@ -142,12 +142,12 @@ WHERE date = '2006-01-02' ORDER BY "ticker", "time"; ---- logical_plan -Sort: data.ticker ASC NULLS LAST, data.time ASC NULLS LAST ---Filter: data.date = Date32("13150") -----TableScan: data projection=[date, ticker, time] +01)Sort: data.ticker ASC NULLS LAST, data.time ASC NULLS LAST +02)--Filter: data.date = Date32("13150") +03)----TableScan: data projection=[date, ticker, time] physical_plan -SortPreservingMergeExec: [ticker@1 ASC NULLS LAST,time@2 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: date@0 = 13150 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST,time@2 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: date@0 = 13150 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 869462b4722a..1acdcde9c8ee 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2004,31 +2004,31 @@ GROUP BY l.col0, l.col1, l.col2 ORDER BY l.col0; ---- logical_plan -Sort: l.col0 ASC NULLS LAST ---Projection: l.col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST] AS last_col1 -----Aggregate: groupBy=[[l.col0, l.col1, l.col2]], aggr=[[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]]] -------Inner Join: l.col0 = r.col0 ---------SubqueryAlias: l -----------TableScan: tab0 projection=[col0, col1, col2] ---------SubqueryAlias: r -----------TableScan: tab0 projection=[col0, col1] +01)Sort: l.col0 ASC NULLS LAST +02)--Projection: l.col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST] AS last_col1 +03)----Aggregate: groupBy=[[l.col0, l.col1, l.col2]], aggr=[[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]]] +04)------Inner Join: l.col0 = r.col0 +05)--------SubqueryAlias: l +06)----------TableScan: tab0 projection=[col0, col1, col2] +07)--------SubqueryAlias: r +08)----------TableScan: tab0 projection=[col0, col1] physical_plan -SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ---SortExec: expr=[col0@0 ASC NULLS LAST] -----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] ---------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] +01)SortPreservingMergeExec: [col0@0 ASC NULLS LAST] +02)--SortExec: expr=[col0@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] +04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +13)------------------------MemoryExec: partitions=1, partition_sizes=[3] +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +16)------------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2067,13 +2067,13 @@ query TT EXPLAIN SELECT a FROM multiple_ordered_table ORDER BY b DESC; ---- logical_plan -Projection: multiple_ordered_table.a ---Sort: multiple_ordered_table.b DESC NULLS FIRST -----TableScan: multiple_ordered_table projection=[a, b] +01)Projection: multiple_ordered_table.a +02)--Sort: multiple_ordered_table.b DESC NULLS FIRST +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a] ---SortExec: expr=[b@1 DESC] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a] +02)--SortExec: expr=[b@1 DESC] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # Final plan shouldn't have SortExec c ASC, # because table already satisfies this ordering. @@ -2081,9 +2081,9 @@ query TT EXPLAIN SELECT a FROM multiple_ordered_table ORDER BY c ASC; ---- logical_plan -Projection: multiple_ordered_table.a ---Sort: multiple_ordered_table.c ASC NULLS LAST -----TableScan: multiple_ordered_table projection=[a, c] +01)Projection: multiple_ordered_table.a +02)--Sort: multiple_ordered_table.c ASC NULLS LAST +03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true # Final plan shouldn't have SortExec a ASC, b ASC, @@ -2092,9 +2092,9 @@ query TT EXPLAIN SELECT a FROM multiple_ordered_table ORDER BY a ASC, b ASC; ---- logical_plan -Projection: multiple_ordered_table.a ---Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST -----TableScan: multiple_ordered_table projection=[a, b] +01)Projection: multiple_ordered_table.a +02)--Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true # test_window_agg_sort @@ -2109,13 +2109,13 @@ EXPLAIN SELECT a, b, GROUP BY b, a ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotated_data_infinite2.c) AS summation1 ---Aggregate: groupBy=[[annotated_data_infinite2.b, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64))]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotated_data_infinite2.c) AS summation1 +02)--Aggregate: groupBy=[[annotated_data_infinite2.b, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64))]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] ---AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] +02)--AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2140,13 +2140,13 @@ EXPLAIN SELECT a, d, GROUP BY d, a ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS summation1 ---Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] -----TableScan: annotated_data_infinite2 projection=[a, c, d] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS summation1 +02)--Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +03)----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan -ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] ---AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) -----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] +01)ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] +02)--AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) +03)----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2173,13 +2173,13 @@ EXPLAIN SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c +02)--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c @@ -2199,13 +2199,13 @@ EXPLAIN SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c +02)--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, LAST_VALUE(c ORDER BY a DESC, c ASC) as last_c @@ -2226,13 +2226,13 @@ EXPLAIN SELECT a, b, LAST_VALUE(c) as last_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) AS last_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c)]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) AS last_c +02)--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c)]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, LAST_VALUE(c) as last_c @@ -2250,11 +2250,11 @@ FROM annotated_data_infinite2 ORDER BY a, b, d; ---- logical_plan -Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST ---TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] +01)Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST +02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -PartialSortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] ---StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +01)PartialSortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +02)--StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT EXPLAIN SELECT * @@ -2263,13 +2263,13 @@ ORDER BY a, b, d LIMIT 50; ---- logical_plan -Limit: skip=0, fetch=50 ---Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST, fetch=50 -----TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] +01)Limit: skip=0, fetch=50 +02)--Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST, fetch=50 +03)----TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -GlobalLimitExec: skip=0, fetch=50 ---PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] -----StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +01)GlobalLimitExec: skip=0, fetch=50 +02)--PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +03)----StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT EXPLAIN SELECT * @@ -2277,11 +2277,11 @@ FROM multiple_ordered_table ORDER BY a, b, d; ---- logical_plan -Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST ---TableScan: multiple_ordered_table projection=[a0, a, b, c, d] +01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST +02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan -SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +01)SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY d) @@ -2289,12 +2289,12 @@ FROM annotated_data_infinite2 GROUP BY a, b; ---- logical_plan -Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]]] ---TableScan: annotated_data_infinite2 projection=[a, b, d] +01)Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]]] +02)--TableScan: annotated_data_infinite2 projection=[a, b, d] physical_plan -AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_infinite2.d)], ordering_mode=Sorted ---PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] -----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] +01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_infinite2.d)], ordering_mode=Sorted +02)--PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] # as can be seen in the result below d is indeed ordered. query II? @@ -2459,14 +2459,14 @@ EXPLAIN SELECT country, (ARRAY_AGG(amount ORDER BY amount ASC)) AS amounts GROUP BY country ---- logical_plan -Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts ---Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] +03)----SortExec: expr=[amount@1 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T? @@ -2488,15 +2488,15 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] +03)----SortExec: expr=[amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2531,16 +2531,16 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted +03)----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2567,16 +2567,16 @@ EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) GROUP BY s.country, s.zip_code ---- logical_plan -Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[zip_code, country, amount] +01)Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[zip_code, country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) -----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) +03)----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TI?R rowsort SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2603,16 +2603,16 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted +03)----SortExec: expr=[country@0 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2638,16 +2638,16 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC, s.amount D GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted +03)----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC, s.amount DESC) AS amounts, @@ -2672,14 +2672,14 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, GROUP BY country ---- logical_plan -Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +03)----SortExec: expr=[amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2703,14 +2703,14 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, GROUP BY country ---- logical_plan -Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +03)----SortExec: expr=[amount@1 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2735,14 +2735,14 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, GROUP BY country ---- logical_plan -Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts ---Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +03)----SortExec: expr=[amount@1 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2765,14 +2765,14 @@ EXPLAIN SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, GROUP BY country ---- logical_plan -Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts ---Aggregate: groupBy=[[sales_global.country]], aggr=[[SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[country, ts, amount] +01)Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@2 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] +03)----SortExec: expr=[amount@2 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2799,14 +2799,14 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, GROUP BY country ---- logical_plan -Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----Sort: sales_global.ts ASC NULLS LAST -------TableScan: sales_global projection=[country, ts, amount] +01)Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----Sort: sales_global.ts ASC NULLS LAST +04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2833,13 +2833,13 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, GROUP BY country ---- logical_plan -Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----TableScan: sales_global projection=[country, ts, amount] +01)Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2862,24 +2862,24 @@ GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency ORDER BY s.sn ---- logical_plan -Sort: s.sn ASC NULLS LAST ---Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST] AS last_rate -----Aggregate: groupBy=[[s.sn, s.zip_code, s.country, s.ts, s.currency]], aggr=[[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]]] -------Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, e.sn, e.amount ---------Inner Join: s.currency = e.currency Filter: s.ts >= e.ts -----------SubqueryAlias: s -------------TableScan: sales_global projection=[zip_code, country, sn, ts, currency] -----------SubqueryAlias: e -------------TableScan: sales_global projection=[sn, ts, currency, amount] +01)Sort: s.sn ASC NULLS LAST +02)--Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST] AS last_rate +03)----Aggregate: groupBy=[[s.sn, s.zip_code, s.country, s.ts, s.currency]], aggr=[[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]]] +04)------Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, e.sn, e.amount +05)--------Inner Join: s.currency = e.currency Filter: s.ts >= e.ts +06)----------SubqueryAlias: s +07)------------TableScan: sales_global projection=[zip_code, country, sn, ts, currency] +08)----------SubqueryAlias: e +09)------------TableScan: sales_global projection=[sn, ts, currency, amount] physical_plan -SortExec: expr=[sn@2 ASC NULLS LAST] ---ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] -----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] -------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] -------------MemoryExec: partitions=1, partition_sizes=[1] -------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[sn@2 ASC NULLS LAST] +02)--ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] +03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] +04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2911,20 +2911,20 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] -------TableScan: sales_global projection=[country, ts, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] +04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2947,20 +2947,20 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -------TableScan: sales_global projection=[country, ts, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2986,16 +2986,16 @@ EXPLAIN SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, FROM sales_global ---- logical_plan -Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 ---Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 +02)--Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3012,16 +3012,16 @@ EXPLAIN SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, FROM sales_global ---- logical_plan -Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 ---Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 +02)--Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3036,17 +3036,17 @@ EXPLAIN SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 FROM sales_global ---- logical_plan -Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +05)--------SortExec: expr=[ts@0 ASC NULLS LAST] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -3060,17 +3060,17 @@ EXPLAIN SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 FROM sales_global ---- logical_plan -Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[ts@0 DESC] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +05)--------SortExec: expr=[ts@0 DESC] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -3084,17 +3084,17 @@ EXPLAIN SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 FROM sales_global ---- logical_plan -Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[amount] +01)Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[amount@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +05)--------SortExec: expr=[amount@0 ASC NULLS LAST] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3110,21 +3110,21 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -------TableScan: sales_global projection=[country, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +04)------TableScan: sales_global projection=[country, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ---------------SortExec: expr=[amount@1 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] +08)--------------SortExec: expr=[amount@1 ASC NULLS LAST] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------MemoryExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3146,21 +3146,21 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -------TableScan: sales_global projection=[country, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +04)------TableScan: sales_global projection=[country, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------------SortExec: expr=[amount@1 DESC] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +08)--------------SortExec: expr=[amount@1 DESC] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3346,20 +3346,20 @@ EXPLAIN SELECT s.sn, s.amount, 2*s.sn ORDER BY sn ---- logical_plan -Sort: s.sn ASC NULLS LAST ---Projection: s.sn, s.amount, Int64(2) * CAST(s.sn AS Int64) -----Aggregate: groupBy=[[s.sn, s.amount]], aggr=[[]] -------SubqueryAlias: s ---------TableScan: sales_global_with_pk projection=[sn, amount] +01)Sort: s.sn ASC NULLS LAST +02)--Projection: s.sn, s.amount, Int64(2) * CAST(s.sn AS Int64) +03)----Aggregate: groupBy=[[s.sn, s.amount]], aggr=[[]] +04)------SubqueryAlias: s +05)--------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -SortPreservingMergeExec: [sn@0 ASC NULLS LAST] ---SortExec: expr=[sn@0 ASC NULLS LAST] -----ProjectionExec: expr=[sn@0 as sn, amount@1 as amount, 2 * CAST(sn@0 AS Int64) as Int64(2) * s.sn] -------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] ---------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +02)--SortExec: expr=[sn@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[sn@0 as sn, amount@1 as amount, 2 * CAST(sn@0 AS Int64) as Int64(2) * s.sn] +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +08)--------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] query IRI SELECT s.sn, s.amount, 2*s.sn @@ -3410,28 +3410,28 @@ EXPLAIN SELECT r.sn, SUM(l.amount), r.amount ORDER BY r.sn ---- logical_plan -Sort: r.sn ASC NULLS LAST ---Projection: r.sn, SUM(l.amount), r.amount -----Aggregate: groupBy=[[r.sn, r.amount]], aggr=[[SUM(CAST(l.amount AS Float64))]] -------Projection: l.amount, r.sn, r.amount ---------Inner Join: Filter: l.sn >= r.sn -----------SubqueryAlias: l -------------TableScan: sales_global_with_pk projection=[sn, amount] -----------SubqueryAlias: r -------------TableScan: sales_global_with_pk projection=[sn, amount] +01)Sort: r.sn ASC NULLS LAST +02)--Projection: r.sn, SUM(l.amount), r.amount +03)----Aggregate: groupBy=[[r.sn, r.amount]], aggr=[[SUM(CAST(l.amount AS Float64))]] +04)------Projection: l.amount, r.sn, r.amount +05)--------Inner Join: Filter: l.sn >= r.sn +06)----------SubqueryAlias: l +07)------------TableScan: sales_global_with_pk projection=[sn, amount] +08)----------SubqueryAlias: r +09)------------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -SortPreservingMergeExec: [sn@0 ASC NULLS LAST] ---SortExec: expr=[sn@0 ASC NULLS LAST] -----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] -------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[SUM(l.amount)] ---------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] -----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 -------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] -------------------CoalescePartitionsExec ---------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +02)--SortExec: expr=[sn@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[SUM(l.amount)] +08)--------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] +09)----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 +10)------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +11)------------------CoalescePartitionsExec +12)--------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] query IRR SELECT r.sn, SUM(l.amount), r.amount @@ -3559,27 +3559,27 @@ EXPLAIN SELECT * ORDER BY l.sn ---- logical_plan -Sort: l.sn ASC NULLS LAST ---Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, l.sum_amount -----Aggregate: groupBy=[[l.sn, l.zip_code, l.country, l.ts, l.currency, l.amount, l.sum_amount]], aggr=[[]] -------SubqueryAlias: l ---------Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum_amount -----------WindowAggr: windowExpr=[[SUM(CAST(l.amount AS Float64)) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -------------SubqueryAlias: l ---------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] +01)Sort: l.sn ASC NULLS LAST +02)--Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, l.sum_amount +03)----Aggregate: groupBy=[[l.sn, l.zip_code, l.country, l.ts, l.currency, l.amount, l.sum_amount]], aggr=[[]] +04)------SubqueryAlias: l +05)--------Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum_amount +06)----------WindowAggr: windowExpr=[[SUM(CAST(l.amount AS Float64)) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +07)------------SubqueryAlias: l +08)--------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] physical_plan -SortPreservingMergeExec: [sn@2 ASC NULLS LAST] ---SortExec: expr=[sn@2 ASC NULLS LAST] -----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] -------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] -------------------BoundedWindowAggExec: wdw=[SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------------CoalescePartitionsExec -----------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +01)SortPreservingMergeExec: [sn@2 ASC NULLS LAST] +02)--SortExec: expr=[sn@2 ASC NULLS LAST] +03)----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] +10)------------------BoundedWindowAggExec: wdw=[SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +11)--------------------CoalescePartitionsExec +12)----------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] query ITIPTRR @@ -3755,14 +3755,14 @@ EXPLAIN SELECT LAST_VALUE(x) FROM FOO; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(foo.x)]] ---TableScan: foo projection=[x] +01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(foo.x)]] +02)--TableScan: foo projection=[x] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(foo.x)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(foo.x)] -------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(foo.x)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(foo.x)] +04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT FIRST_VALUE(x) @@ -3777,14 +3777,14 @@ EXPLAIN SELECT FIRST_VALUE(x) FROM FOO; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(foo.x)]] ---TableScan: foo projection=[x] +01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(foo.x)]] +02)--TableScan: foo projection=[x] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(foo.x)] -------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(foo.x)] +04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] # Since both ordering requirements are satisfied, there shouldn't be # any SortExec in the final plan. @@ -3795,17 +3795,17 @@ FROM multiple_ordered_table GROUP BY d; ---- logical_plan -Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST] AS first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] AS last_c ---Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] -----TableScan: multiple_ordered_table projection=[a, c, d] +01)Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST] AS first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] AS last_c +02)--Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] +03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -3837,8 +3837,8 @@ FROM multiple_ordered_table ORDER BY c ASC; ---- logical_plan -Sort: multiple_ordered_table.c ASC NULLS LAST ---TableScan: multiple_ordered_table projection=[c] +01)Sort: multiple_ordered_table.c ASC NULLS LAST +02)--TableScan: multiple_ordered_table projection=[c] physical_plan 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 statement ok @@ -3856,27 +3856,27 @@ GROUP BY row_n ORDER BY row_n ---- logical_plan -Projection: amount_usd ---Sort: row_n ASC NULLS LAST -----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n -------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] ---------Projection: l.a, l.d, row_n -----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) -------------SubqueryAlias: l ---------------TableScan: multiple_ordered_table projection=[a, d] -------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n ---------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------------SubqueryAlias: r -------------------TableScan: multiple_ordered_table projection=[a, d] +01)Projection: amount_usd +02)--Sort: row_n ASC NULLS LAST +03)----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +04)------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +05)--------Projection: l.a, l.d, row_n +06)----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +07)------------SubqueryAlias: l +08)--------------TableScan: multiple_ordered_table projection=[a, d] +09)------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +10)--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +11)----------------SubqueryAlias: r +12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] ---AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true ---------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # reset partition number to 8. statement ok @@ -3908,16 +3908,16 @@ FROM multiple_ordered_table_with_pk GROUP BY c; ---- logical_plan -Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] ---TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +02)--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ---SortExec: expr=[c@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +02)--SortExec: expr=[c@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true # drop table multiple_ordered_table_with_pk statement ok @@ -3949,16 +3949,16 @@ FROM multiple_ordered_table_with_pk GROUP BY c; ---- logical_plan -Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] ---TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +02)--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ---SortExec: expr=[c@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +02)--SortExec: expr=[c@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3972,15 +3972,15 @@ EXPLAIN SELECT c, sum1 GROUP BY c; ---- logical_plan -Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, sum1]], aggr=[[]] ---Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[c, d] +01)Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, sum1]], aggr=[[]] +02)--Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[c, d] physical_plan -AggregateExec: mode=Single, gby=[c@0 as c, sum1@1 as sum1], aggr=[], ordering_mode=PartiallySorted([0]) ---ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] -----AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)AggregateExec: mode=Single, gby=[c@0 as c, sum1@1 as sum1], aggr=[], ordering_mode=PartiallySorted([0]) +02)--ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +03)----AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb @@ -3990,17 +3990,17 @@ EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb GROUP BY c); ---- logical_plan -Projection: multiple_ordered_table_with_pk.c, sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sumb ---WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table_with_pk.b AS Int64)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 -------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] ---------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)Projection: multiple_ordered_table_with_pk.c, sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sumb +02)--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table_with_pk.b AS Int64)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +04)------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +05)--------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as sumb] ---WindowAggExec: wdw=[SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -----ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as sumb] +02)--WindowAggExec: wdw=[SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +03)----ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +04)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4015,26 +4015,26 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 ON lhs.b=rhs.b; ---- logical_plan -Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---Inner Join: lhs.b = rhs.b -----SubqueryAlias: lhs -------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] -----SubqueryAlias: rhs -------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 +02)--Inner Join: lhs.b = rhs.b +03)----SubqueryAlias: lhs +04)------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +05)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +06)----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +07)----SubqueryAlias: rhs +08)------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +09)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +10)----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] -------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true -------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] +04)------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +07)------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +08)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +09)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4048,25 +4048,25 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 GROUP BY c) as rhs; ---- logical_plan -Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---CrossJoin: -----SubqueryAlias: lhs -------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----------TableScan: multiple_ordered_table_with_pk projection=[c, d] -----SubqueryAlias: rhs -------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----------TableScan: multiple_ordered_table_with_pk projection=[c, d] +01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 +02)--CrossJoin: +03)----SubqueryAlias: lhs +04)------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +05)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +06)----------TableScan: multiple_ordered_table_with_pk projection=[c, d] +07)----SubqueryAlias: rhs +08)------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +09)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +10)----------TableScan: multiple_ordered_table_with_pk projection=[c, d] physical_plan -ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] ---CrossJoinExec -----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] -------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true -----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] -------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] +02)--CrossJoinExec +03)----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +04)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +06)----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +07)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +08)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true # we do not generate physical plan for Repartition yet (e.g Distribute By queries). query TT @@ -4077,10 +4077,10 @@ FROM (SELECT c, b, a, SUM(d) as sum1 DISTRIBUTE BY a ---- logical_plan -Repartition: DistributeBy(a) ---Projection: multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, b, c, d] +01)Repartition: DistributeBy(a) +02)--Projection: multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[a, b, c, d] # union with aggregate query TT @@ -4093,21 +4093,21 @@ UNION ALL GROUP BY c ---- logical_plan -Union ---Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] ---Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +01)Union +02)--Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +05)--Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +06)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +07)------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] physical_plan -UnionExec ---ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true ---ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)UnionExec +02)--ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +03)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +05)--ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +06)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +07)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true # table scan should be simplified. query TT @@ -4116,13 +4116,13 @@ EXPLAIN SELECT c, a, SUM(d) as sum1 GROUP BY c ---- logical_plan -Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +01)Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +02)--Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +03)----TableScan: multiple_ordered_table_with_pk projection=[a, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true # limit should be simplified query TT @@ -4133,15 +4133,15 @@ EXPLAIN SELECT * LIMIT 5) ---- logical_plan -Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---Limit: skip=0, fetch=5 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +01)Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +02)--Limit: skip=0, fetch=5 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---GlobalLimitExec: skip=0, fetch=5 -----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 8; @@ -4169,40 +4169,40 @@ query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT x) FROM t1 GROUP BY y; ---- logical_plan -Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) ---Aggregate: groupBy=[[t1.y]], aggr=[[SUM(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] -----TableScan: t1 projection=[x, y] +01)Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) +02)--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] +03)----TableScan: t1 projection=[x, y] physical_plan -ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] ---AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +07)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; ---- logical_plan -Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) ---Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] -----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] -------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y ---------TableScan: t1 projection=[x, y] +01)Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) +02)--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] +03)----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] +04)------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y +05)--------TableScan: t1 projection=[x, y] physical_plan -ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] ---AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] -----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] -------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] ---------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] -----------------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] +11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] +12)----------------------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok @@ -4223,21 +4223,21 @@ EXPLAIN SELECT date_bin('15 minutes', ts) as time_chunks LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: time_chunks DESC NULLS FIRST, fetch=5 -----Projection: date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts) AS time_chunks -------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)]], aggr=[[]] ---------TableScan: unbounded_csv_with_timestamps projection=[ts] +01)Limit: skip=0, fetch=5 +02)--Sort: time_chunks DESC NULLS FIRST, fetch=5 +03)----Projection: date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts) AS time_chunks +04)------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)]], aggr=[[]] +05)--------TableScan: unbounded_csv_with_timestamps projection=[ts] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 -----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] -------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC -------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 +03)----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] +04)------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC +07)------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] query P SELECT date_bin('15 minutes', ts) as time_chunks @@ -4278,22 +4278,22 @@ EXPLAIN SELECT extract(month from ts) as months LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: months DESC NULLS FIRST, fetch=5 -----Projection: date_part(Utf8("MONTH"),csv_with_timestamps.ts) AS months -------Aggregate: groupBy=[[date_part(Utf8("MONTH"), csv_with_timestamps.ts)]], aggr=[[]] ---------TableScan: csv_with_timestamps projection=[ts] +01)Limit: skip=0, fetch=5 +02)--Sort: months DESC NULLS FIRST, fetch=5 +03)----Projection: date_part(Utf8("MONTH"),csv_with_timestamps.ts) AS months +04)------Aggregate: groupBy=[[date_part(Utf8("MONTH"), csv_with_timestamps.ts)]], aggr=[[]] +05)--------TableScan: csv_with_timestamps projection=[ts] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [months@0 DESC], fetch=5 -----SortExec: TopK(fetch=5), expr=[months@0 DESC] -------ProjectionExec: expr=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as months] ---------AggregateExec: mode=FinalPartitioned, gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 ---------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [months@0 DESC], fetch=5 +03)----SortExec: TopK(fetch=5), expr=[months@0 DESC] +04)------ProjectionExec: expr=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as months] +05)--------AggregateExec: mode=FinalPartitioned, gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 +08)--------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false query R SELECT extract(month from ts) as months @@ -4326,16 +4326,16 @@ EXPLAIN SELECT name, date_bin('15 minutes', ts) as time_chunks LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: unbounded_csv_with_timestamps2.name DESC NULLS FIRST, time_chunks DESC NULLS FIRST, fetch=5 -----Projection: unbounded_csv_with_timestamps2.name, date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps2.ts) AS time_chunks -------TableScan: unbounded_csv_with_timestamps2 projection=[name, ts] +01)Limit: skip=0, fetch=5 +02)--Sort: unbounded_csv_with_timestamps2.name DESC NULLS FIRST, time_chunks DESC NULLS FIRST, fetch=5 +03)----Projection: unbounded_csv_with_timestamps2.name, date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps2.ts) AS time_chunks +04)------TableScan: unbounded_csv_with_timestamps2 projection=[name, ts] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [name@0 DESC,time_chunks@1 DESC], fetch=5 -----ProjectionExec: expr=[name@0 as name, date_bin(900000000000, ts@1) as time_chunks] -------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [name@0 DESC,time_chunks@1 DESC], fetch=5 +03)----ProjectionExec: expr=[name@0 as name, date_bin(900000000000, ts@1) as time_chunks] +04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok drop table t1 @@ -4393,25 +4393,25 @@ query TT EXPLAIN SELECT c1, count(distinct c2), min(distinct c2), sum(c3), max(c4) FROM aggregate_test_100 GROUP BY c1 ORDER BY c1; ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, COUNT(alias1) AS COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1) AS MIN(DISTINCT aggregate_test_100.c2), SUM(alias2) AS SUM(aggregate_test_100.c3), MAX(alias3) AS MAX(aggregate_test_100.c4) -----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)]] -------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[SUM(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, MAX(aggregate_test_100.c4) AS alias3]] ---------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, COUNT(alias1) AS COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1) AS MIN(DISTINCT aggregate_test_100.c2), SUM(alias2) AS SUM(aggregate_test_100.c3), MAX(alias3) AS MAX(aggregate_test_100.c4) +03)----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)]] +04)------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[SUM(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, MAX(aggregate_test_100.c4) AS alias3]] +05)--------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ---SortExec: expr=[c1@0 ASC NULLS LAST] -----ProjectionExec: expr=[c1@0 as c1, COUNT(alias1)@1 as COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1)@2 as MIN(DISTINCT aggregate_test_100.c2), SUM(alias2)@3 as SUM(aggregate_test_100.c3), MAX(alias3)@4 as MAX(aggregate_test_100.c4)] -------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] ---------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 ---------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] -----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +02)--SortExec: expr=[c1@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c1@0 as c1, COUNT(alias1)@1 as COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1)@2 as MIN(DISTINCT aggregate_test_100.c2), SUM(alias2)@3 as SUM(aggregate_test_100.c3), MAX(alias3)@4 as MAX(aggregate_test_100.c4)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 +11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true # Use PostgreSQL dialect statement ok @@ -4573,20 +4573,20 @@ ORDER BY MAX(t1) DESC LIMIT 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: MAX(timestamp_table.t1) DESC NULLS FIRST, fetch=4 -----Aggregate: groupBy=[[timestamp_table.c2]], aggr=[[MAX(timestamp_table.t1)]] -------TableScan: timestamp_table projection=[t1, c2] +01)Limit: skip=0, fetch=4 +02)--Sort: MAX(timestamp_table.t1) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[timestamp_table.c2]], aggr=[[MAX(timestamp_table.t1)]] +04)------TableScan: timestamp_table projection=[t1, c2] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [MAX(timestamp_table.t1)@1 DESC], fetch=4 -----SortExec: TopK(fetch=4), expr=[MAX(timestamp_table.t1)@1 DESC] -------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 -----------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [MAX(timestamp_table.t1)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[MAX(timestamp_table.t1)@1 DESC] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 +09)----------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true # Clean up statement ok @@ -4970,12 +4970,12 @@ GROUP BY a, b ORDER BY a, b; ---- logical_plan -Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] -----TableScan: multiple_ordered_table projection=[a, b, c] +01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] +03)----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan -AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.c)], ordering_mode=Sorted ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.c)], ordering_mode=Sorted +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query II? SELECT a, b, ARRAY_AGG(c ORDER BY c DESC) diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index b16a169598e7..e20e31308ed0 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -57,23 +57,23 @@ FROM aggregate_test_100 ORDER by c1 ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 -----Sort: aggregate_test_100.c1 ASC NULLS LAST -------Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 ---------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -----------TableScan: aggregate_test_100 projection=[c1, c4, c9] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST +04)------Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 +05)--------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +06)----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -DataSinkExec: sink=MemoryTable (partitions=1) ---ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] -----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] -------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] ---------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +01)DataSinkExec: sink=MemoryTable (partitions=1) +02)--ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] +03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] +05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true query II INSERT INTO table_without_values SELECT @@ -120,20 +120,20 @@ COUNT(*) OVER(PARTITION BY c1 ORDER BY c9 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWI FROM aggregate_test_100 ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 -----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -------TableScan: aggregate_test_100 projection=[c1, c4, c9] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +03)----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +04)------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -DataSinkExec: sink=MemoryTable (partitions=1) ---CoalescePartitionsExec -----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +01)DataSinkExec: sink=MemoryTable (partitions=1) +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true @@ -168,23 +168,23 @@ FROM aggregate_test_100 ORDER BY c1 ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: a1 AS a1, a2 AS a2 -----Sort: aggregate_test_100.c1 ASC NULLS LAST -------Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a2, aggregate_test_100.c1 ---------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -----------TableScan: aggregate_test_100 projection=[c1, c4, c9] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: a1 AS a1, a2 AS a2 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST +04)------Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS a2, aggregate_test_100.c1 +05)--------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +06)----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -DataSinkExec: sink=MemoryTable (partitions=8) ---ProjectionExec: expr=[a1@0 as a1, a2@1 as a2] -----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] -------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] ---------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +01)DataSinkExec: sink=MemoryTable (partitions=8) +02)--ProjectionExec: expr=[a1@0 as a1, a2@1 as a2] +03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] +05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true query II @@ -212,14 +212,14 @@ query TT explain insert into table_without_values select c1 from aggregate_test_100 order by c1; ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: aggregate_test_100.c1 AS c1 -----Sort: aggregate_test_100.c1 ASC NULLS LAST -------TableScan: aggregate_test_100 projection=[c1] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: aggregate_test_100.c1 AS c1 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST +04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -DataSinkExec: sink=MemoryTable (partitions=1) ---SortExec: expr=[c1@0 ASC NULLS LAST] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +01)DataSinkExec: sink=MemoryTable (partitions=1) +02)--SortExec: expr=[c1@0 ASC NULLS LAST] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query T insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 0033b070ec1a..65b57b61de39 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -121,14 +121,14 @@ query TT EXPLAIN INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); ---- logical_plan -Dml: op=[Insert Into] table=[ordered_insert_test] ---Projection: column1 AS a, column2 AS b -----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... +01)Dml: op=[Insert Into] table=[ordered_insert_test] +02)--Projection: column1 AS a, column2 AS b +03)----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... physical_plan -DataSinkExec: sink=CsvSink(file_groups=[]) ---SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] -----ProjectionExec: expr=[column1@0 as a, column2@1 as b] -------ValuesExec +01)DataSinkExec: sink=CsvSink(file_groups=[]) +02)--SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] +03)----ProjectionExec: expr=[column1@0 as a, column2@1 as b] +04)------ValuesExec query II INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); @@ -346,23 +346,23 @@ FROM aggregate_test_100 ORDER by c1 ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 -----Sort: aggregate_test_100.c1 ASC NULLS LAST -------Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 ---------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -----------TableScan: aggregate_test_100 projection=[c1, c4, c9] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST +04)------Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, aggregate_test_100.c1 +05)--------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +06)----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -DataSinkExec: sink=ParquetSink(file_groups=[]) ---ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] -----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] -------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] ---------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +01)DataSinkExec: sink=ParquetSink(file_groups=[]) +02)--ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] +03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] +05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true query II INSERT INTO table_without_values SELECT @@ -410,20 +410,20 @@ COUNT(*) OVER(PARTITION BY c1 ORDER BY c9 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWI FROM aggregate_test_100 ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 -----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -------TableScan: aggregate_test_100 projection=[c1, c4, c9] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS field2 +03)----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +04)------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -DataSinkExec: sink=ParquetSink(file_groups=[]) ---CoalescePartitionsExec -----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +01)DataSinkExec: sink=ParquetSink(file_groups=[]) +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true @@ -453,14 +453,14 @@ query TT explain insert into table_without_values select c1 from aggregate_test_100 order by c1; ---- logical_plan -Dml: op=[Insert Into] table=[table_without_values] ---Projection: aggregate_test_100.c1 AS c1 -----Sort: aggregate_test_100.c1 ASC NULLS LAST -------TableScan: aggregate_test_100 projection=[c1] +01)Dml: op=[Insert Into] table=[table_without_values] +02)--Projection: aggregate_test_100.c1 AS c1 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST +04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -DataSinkExec: sink=ParquetSink(file_groups=[]) ---SortExec: expr=[c1@0 ASC NULLS LAST] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +01)DataSinkExec: sink=ParquetSink(file_groups=[]) +02)--SortExec: expr=[c1@0 ASC NULLS LAST] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query T insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index 135ab8075425..f451d2e763fe 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -671,14 +671,14 @@ query TT explain select * from t1 inner join t2 on true; ---- logical_plan -CrossJoin: ---TableScan: t1 projection=[t1_id, t1_name, t1_int] ---TableScan: t2 projection=[t2_id, t2_name, t2_int] +01)CrossJoin: +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan -CrossJoinExec ---MemoryExec: partitions=1, partition_sizes=[1] ---RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----MemoryExec: partitions=1, partition_sizes=[1] +01)CrossJoinExec +02)--MemoryExec: partitions=1, partition_sizes=[1] +03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)----MemoryExec: partitions=1, partition_sizes=[1] statement ok drop table IF EXISTS t1; @@ -749,15 +749,15 @@ t1 as t2 WHERE t1.a=t2.a; ---- logical_plan -Inner Join: t1.a = t2.a ---TableScan: t1 projection=[a, b] ---SubqueryAlias: t2 -----TableScan: t1 projection=[a, b] +01)Inner Join: t1.a = t2.a +02)--TableScan: t1 projection=[a, b] +03)--SubqueryAlias: t2 +04)----TableScan: t1 projection=[a, b] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -----MemoryExec: partitions=1, partition_sizes=[1] -----MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----MemoryExec: partitions=1, partition_sizes=[1] # Reset the configs to old values statement ok diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 61e95815f999..60a14f78bdf5 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -46,22 +46,22 @@ EXPLAIN SELECT t2.a LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: t2.a ASC NULLS LAST, fetch=5 -----Projection: t2.a -------Inner Join: t1.c = t2.c ---------SubqueryAlias: t1 -----------TableScan: annotated_data projection=[c] ---------SubqueryAlias: t2 -----------TableScan: annotated_data projection=[a, c] +01)Limit: skip=0, fetch=5 +02)--Sort: t2.a ASC NULLS LAST, fetch=5 +03)----Projection: t2.a +04)------Inner Join: t1.c = t2.c +05)--------SubqueryAlias: t1 +06)----------TableScan: annotated_data projection=[c] +07)--------SubqueryAlias: t2 +08)----------TableScan: annotated_data projection=[a, c] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 -----CoalesceBatchesExec: target_batch_size=8192 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true +06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true # preserve_inner_join query IIII nosort @@ -87,26 +87,26 @@ EXPLAIN SELECT t2.a as a2, t2.b LIMIT 10 ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: a2 ASC NULLS LAST, t2.b ASC NULLS LAST, fetch=10 -----Projection: t2.a AS a2, t2.b -------RightSemi Join: t1.d = t2.d, t1.c = t2.c ---------SubqueryAlias: t1 -----------TableScan: annotated_data projection=[c, d] ---------SubqueryAlias: t2 -----------Filter: annotated_data.d = Int32(3) -------------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] +01)Limit: skip=0, fetch=10 +02)--Sort: a2 ASC NULLS LAST, t2.b ASC NULLS LAST, fetch=10 +03)----Projection: t2.a AS a2, t2.b +04)------RightSemi Join: t1.d = t2.d, t1.c = t2.c +05)--------SubqueryAlias: t1 +06)----------TableScan: annotated_data projection=[c, d] +07)--------SubqueryAlias: t2 +08)----------Filter: annotated_data.d = Int32(3) +09)------------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 -----ProjectionExec: expr=[a@0 as a2, b@1 as b] -------CoalesceBatchesExec: target_batch_size=8192 ---------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true -----------CoalesceBatchesExec: target_batch_size=8192 -------------FilterExec: d@3 = 3 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 +03)----ProjectionExec: expr=[a@0 as a2, b@1 as b] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +07)----------CoalesceBatchesExec: target_batch_size=8192 +08)------------FilterExec: d@3 = 3 +09)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index b47c0a457fe1..04d6e46caf5e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1037,11 +1037,11 @@ LEFT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t2.t2_id < 100 ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_id < UInt32(100) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_id < UInt32(100) -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_id < UInt32(100) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--Filter: join_t2.t2_id < UInt32(100) +05)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce left join 2 (to inner join) @@ -1053,10 +1053,10 @@ LEFT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t2.t2_int < 10 or (join_t1.t1_int > 2 and join_t2.t2_name != 'w') ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int < UInt32(10) OR join_t1.t1_int > UInt32(2) AND join_t2.t2_name != Utf8("w") ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_int < UInt32(10) OR join_t2.t2_name != Utf8("w") -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int < UInt32(10) OR join_t1.t1_int > UInt32(2) AND join_t2.t2_name != Utf8("w") +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--Filter: join_t2.t2_int < UInt32(10) OR join_t2.t2_name != Utf8("w") +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce left join 3 (to inner join) @@ -1073,16 +1073,16 @@ LEFT JOIN join_t2 on t3.t1_int = join_t2.t2_int WHERE t3.t1_id < 100 ---- logical_plan -Left Join: t3.t1_int = join_t2.t2_int ---SubqueryAlias: t3 -----Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int -------Inner Join: join_t1.t1_id = join_t2.t2_id ---------Filter: join_t1.t1_id < UInt32(100) -----------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---------Projection: join_t2.t2_id -----------Filter: join_t2.t2_int < UInt32(3) AND join_t2.t2_id < UInt32(100) -------------TableScan: join_t2 projection=[t2_id, t2_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Left Join: t3.t1_int = join_t2.t2_int +02)--SubqueryAlias: t3 +03)----Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int +04)------Inner Join: join_t1.t1_id = join_t2.t2_id +05)--------Filter: join_t1.t1_id < UInt32(100) +06)----------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +07)--------Projection: join_t2.t2_id +08)----------Filter: join_t2.t2_int < UInt32(3) AND join_t2.t2_id < UInt32(100) +09)------------TableScan: join_t2 projection=[t2_id, t2_int] +10)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce right join 1 (to inner join) @@ -1094,10 +1094,10 @@ RIGHT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t1.t1_int IS NOT NULL ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_int IS NOT NULL -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_int IS NOT NULL +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce right join 2 (to inner join) @@ -1109,9 +1109,9 @@ RIGHT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE NOT (join_t1.t1_int = join_t2.t2_int) ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int != join_t1.t1_int ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int != join_t1.t1_int +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to right join @@ -1123,10 +1123,10 @@ FULL JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t2.t2_name IS NOT NULL ---- logical_plan -Right Join: join_t1.t1_id = join_t2.t2_id ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_name IS NOT NULL -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Right Join: join_t1.t1_id = join_t2.t2_id +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--Filter: join_t2.t2_name IS NOT NULL +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to left join @@ -1138,10 +1138,10 @@ FULL JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id WHERE join_t1.t1_name != 'b' ---- logical_plan -Left Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_name != Utf8("b") -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Left Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_name != Utf8("b") +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to inner join @@ -1153,11 +1153,11 @@ FULL JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t1.t1_name != 'b' and join_t2.t2_name = 'x' ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_name != Utf8("b") -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_name = Utf8("x") -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_name != Utf8("b") +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--Filter: join_t2.t2_name = Utf8("x") +05)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] ### # Configuration teardown @@ -1205,11 +1205,11 @@ FROM lsaj_t1 LEFT SEMI JOIN lsaj_t2 ON (lsaj_t1.t1_id = lsaj_t2.t2_id and lsaj_t2.t2_int > 1) ---- logical_plan -LeftSemi Join: lsaj_t1.t1_id = lsaj_t2.t2_id ---TableScan: lsaj_t1 projection=[t1_id, t1_name] ---Projection: lsaj_t2.t2_id -----Filter: lsaj_t2.t2_int > UInt32(1) -------TableScan: lsaj_t2 projection=[t2_id, t2_int] +01)LeftSemi Join: lsaj_t1.t1_id = lsaj_t2.t2_id +02)--TableScan: lsaj_t1 projection=[t1_id, t1_name] +03)--Projection: lsaj_t2.t2_id +04)----Filter: lsaj_t2.t2_int > UInt32(1) +05)------TableScan: lsaj_t2 projection=[t2_id, t2_int] # Left anti join @@ -1318,23 +1318,23 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] ---Projection: join_t1.t1_id -----Inner Join: join_t1.t1_id = join_t2.t2_id -------TableScan: join_t1 projection=[t1_id] -------TableScan: join_t2 projection=[t2_id] +01)Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] +02)--Projection: join_t1.t1_id +03)----Inner Join: join_t1.t1_id = join_t2.t2_id +04)------TableScan: join_t1 projection=[t1_id] +05)------TableScan: join_t2 projection=[t2_id] physical_plan -AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1343,25 +1343,25 @@ from (select * from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id) group by t1_id ---- logical_plan -Projection: COUNT(*) ---Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----Projection: join_t1.t1_id -------Inner Join: join_t1.t1_id = join_t2.t2_id ---------TableScan: join_t1 projection=[t1_id] ---------TableScan: join_t2 projection=[t2_id] +01)Projection: COUNT(*) +02)--Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----Projection: join_t1.t1_id +04)------Inner Join: join_t1.t1_id = join_t2.t2_id +05)--------TableScan: join_t1 projection=[t1_id] +06)--------TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] ---AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] +02)--AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1370,30 +1370,30 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) ---Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] -----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] -------Projection: join_t1.t1_id ---------Inner Join: join_t1.t1_id = join_t2.t2_id -----------TableScan: join_t1 projection=[t1_id] -----------TableScan: join_t2 projection=[t2_id] +01)Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) +02)--Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] +03)----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] +04)------Projection: join_t1.t1_id +05)--------Inner Join: join_t1.t1_id = join_t2.t2_id +06)----------TableScan: join_t1 projection=[t1_id] +07)----------TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] ---AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] ---------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] -----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -------------CoalesceBatchesExec: target_batch_size=2 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[1] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] +02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +13)----------------CoalesceBatchesExec: target_batch_size=2 +14)------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +15)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)----------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1408,9 +1408,9 @@ cross join join_t2 where join_t1.t1_id + 12 = join_t2.t2_id + 1 ---- logical_plan -Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = CAST(join_t2.t2_id AS Int64) + Int64(1) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = CAST(join_t2.t2_id AS Int64) + Int64(1) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce cross join with cast expr join key (to inner join) @@ -1421,10 +1421,10 @@ from join_t1 cross join join_t2 where join_t1.t1_id + 11 = cast(join_t2.t2_id as BIGINT) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] ##### @@ -1446,21 +1446,21 @@ from join_t1, join_t2 where join_t1.t1_id + 11 = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) ---Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) +02)--Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +04)------CoalescePartitionsExec +05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1472,24 +1472,24 @@ from join_t1, join_t2 where join_t1.t1_id + 11 = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) ---Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) +02)--Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 +06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 +11)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)--------------MemoryExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1503,21 +1503,21 @@ FROM join_t1 INNER JOIN join_t2 ON join_t1.t1_id + cast(12 as INT UNSIGNED) = join_t2.t2_id + cast(1 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalescePartitionsExec +05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1529,24 +1529,24 @@ FROM join_t1 INNER JOIN join_t2 ON join_t1.t1_id + cast(12 as INT UNSIGNED) = join_t2.t2_id + cast(1 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 +06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 +11)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)--------------MemoryExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1561,18 +1561,18 @@ INNER JOIN join_t2 ON join_t1.t1_id + cast(11 as INT UNSIGNED) = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1585,23 +1585,23 @@ INNER JOIN join_t2 ON join_t1.t1_id + cast(11 as INT UNSIGNED) = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 +10)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1616,20 +1616,20 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalescePartitionsExec +05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1642,23 +1642,23 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 +06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)------------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1673,16 +1673,16 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -----MemoryExec: partitions=1, partition_sizes=[1] -----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1695,21 +1695,21 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 +09)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -1728,9 +1728,9 @@ from join_t1 inner join join_t2 on join_t1.t1_id + 11 = join_t2.t2_id ---- logical_plan -Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) ---TableScan: join_t1 projection=[t1_id, t1_name] ---TableScan: join_t2 projection=[t2_id] +01)Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +02)--TableScan: join_t1 projection=[t1_id, t1_name] +03)--TableScan: join_t2 projection=[t2_id] # Join only with filter @@ -1741,9 +1741,9 @@ from join_t1 inner join join_t2 on join_t1.t1_id * 4 < join_t2.t2_id ---- logical_plan -Inner Join: Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) ---TableScan: join_t1 projection=[t1_id, t1_name] ---TableScan: join_t2 projection=[t2_id] +01)Inner Join: Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) +02)--TableScan: join_t1 projection=[t1_id, t1_name] +03)--TableScan: join_t2 projection=[t2_id] # Type coercion join with filter and equi expr @@ -1755,9 +1755,9 @@ inner join join_t2 on join_t1.t1_id * 5 = join_t2.t2_id and join_t1.t1_id * 4 < join_t2.t2_id ---- logical_plan -Inner Join: CAST(join_t1.t1_id AS Int64) * Int64(5) = CAST(join_t2.t2_id AS Int64) Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) ---TableScan: join_t1 projection=[t1_id, t1_name] ---TableScan: join_t2 projection=[t2_id] +01)Inner Join: CAST(join_t1.t1_id AS Int64) * Int64(5) = CAST(join_t2.t2_id AS Int64) Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) +02)--TableScan: join_t1 projection=[t1_id, t1_name] +03)--TableScan: join_t2 projection=[t2_id] # Test cross join to groupby with different key ordering @@ -1810,11 +1810,11 @@ from join_t1 where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +05)------TableScan: join_t2 projection=[t2_id] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1839,12 +1839,12 @@ where join_t1.t1_id + 12 in ) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int -------Filter: join_t2.t2_int > UInt32(0) ---------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int +05)------Filter: join_t2.t2_int > UInt32(0) +06)--------TableScan: join_t2 projection=[t2_id, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1875,12 +1875,12 @@ where join_t1.t1_id + 12 in ) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name -------Filter: join_t2.t2_int > UInt32(0) ---------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name +05)------Filter: join_t2.t2_int > UInt32(0) +06)--------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1907,12 +1907,12 @@ where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2 where join_t1.t1_int > 0) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) ---Filter: join_t1.t1_int > UInt32(0) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) +02)--Filter: join_t1.t1_int > UInt32(0) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--SubqueryAlias: __correlated_sq_1 +05)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +06)------TableScan: join_t2 projection=[t2_id] # Not in subquery to join with correlated outer filter @@ -1924,11 +1924,11 @@ where join_t1.t1_id + 12 not in (select join_t2.t2_id + 1 from join_t2 where join_t1.t1_int > 0) ---- logical_plan -LeftAnti Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int > UInt32(0) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_id] +01)LeftAnti Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int > UInt32(0) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +05)------TableScan: join_t2 projection=[t2_id] # In subquery to join with outer filter @@ -1946,12 +1946,12 @@ where join_t1.t1_id + 12 in and join_t1.t1_id > 0 ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name ---Filter: join_t1.t1_id > UInt32(0) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name -------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name +02)--Filter: join_t1.t1_id > UInt32(0) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--SubqueryAlias: __correlated_sq_1 +05)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name +06)------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1979,16 +1979,16 @@ where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2) and join_t1.t1_id > 0 ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_int AS Int64) = __correlated_sq_2.join_t2.t2_int + Int64(1) ---LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) -----Filter: join_t1.t1_id > UInt32(0) -------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -----SubqueryAlias: __correlated_sq_1 -------Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) ---------TableScan: join_t2 projection=[t2_id] ---SubqueryAlias: __correlated_sq_2 -----Projection: CAST(join_t2.t2_int AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_int] +01)LeftSemi Join: CAST(join_t1.t1_int AS Int64) = __correlated_sq_2.join_t2.t2_int + Int64(1) +02)--LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) +03)----Filter: join_t1.t1_id > UInt32(0) +04)------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +05)----SubqueryAlias: __correlated_sq_1 +06)------Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +07)--------TableScan: join_t2 projection=[t2_id] +08)--SubqueryAlias: __correlated_sq_2 +09)----Projection: CAST(join_t2.t2_int AS Int64) + Int64(1) +10)------TableScan: join_t2 projection=[t2_int] query ITI select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -2020,24 +2020,24 @@ INNER JOIN join_t2 ON join_t1.t1_id > join_t2.t2_id WHERE join_t1.t1_id > 10 AND join_t2.t2_int > 1 ---- logical_plan -Inner Join: Filter: join_t1.t1_id > join_t2.t2_id ---Filter: join_t1.t1_id > UInt32(10) -----TableScan: join_t1 projection=[t1_id] ---Projection: join_t2.t2_id -----Filter: join_t2.t2_int > UInt32(1) -------TableScan: join_t2 projection=[t2_id, t2_int] +01)Inner Join: Filter: join_t1.t1_id > join_t2.t2_id +02)--Filter: join_t1.t1_id > UInt32(10) +03)----TableScan: join_t1 projection=[t1_id] +04)--Projection: join_t2.t2_id +05)----Filter: join_t2.t2_int > UInt32(1) +06)------TableScan: join_t2 projection=[t2_id, t2_int] physical_plan -NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 ---CoalesceBatchesExec: target_batch_size=2 -----FilterExec: t1_id@0 > 10 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] ---CoalescePartitionsExec -----ProjectionExec: expr=[t2_id@0 as t2_id] -------CoalesceBatchesExec: target_batch_size=2 ---------FilterExec: t2_int@1 > 1 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 +02)--CoalesceBatchesExec: target_batch_size=2 +03)----FilterExec: t1_id@0 > 10 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--CoalescePartitionsExec +07)----ProjectionExec: expr=[t2_id@0 as t2_id] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------FilterExec: t2_int@1 > 1 +10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2059,24 +2059,24 @@ RIGHT JOIN (select t2_id from join_t2 where join_t2.t2_id > 11) as join_t2 ON join_t1.t1_id < join_t2.t2_id ---- logical_plan -Right Join: Filter: join_t1.t1_id < join_t2.t2_id ---SubqueryAlias: join_t1 -----Filter: join_t1.t1_id > UInt32(22) -------TableScan: join_t1 projection=[t1_id] ---SubqueryAlias: join_t2 -----Filter: join_t2.t2_id > UInt32(11) -------TableScan: join_t2 projection=[t2_id] +01)Right Join: Filter: join_t1.t1_id < join_t2.t2_id +02)--SubqueryAlias: join_t1 +03)----Filter: join_t1.t1_id > UInt32(22) +04)------TableScan: join_t1 projection=[t1_id] +05)--SubqueryAlias: join_t2 +06)----Filter: join_t2.t2_id > UInt32(11) +07)------TableScan: join_t2 projection=[t2_id] physical_plan -NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 ---CoalescePartitionsExec -----CoalesceBatchesExec: target_batch_size=2 -------FilterExec: t1_id@0 > 22 ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] ---CoalesceBatchesExec: target_batch_size=2 -----FilterExec: t2_id@0 > 11 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 +02)--CoalescePartitionsExec +03)----CoalesceBatchesExec: target_batch_size=2 +04)------FilterExec: t1_id@0 > 22 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)--CoalesceBatchesExec: target_batch_size=2 +08)----FilterExec: t2_id@0 > 11 +09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2113,10 +2113,10 @@ WHERE EXISTS ( ) ---- logical_plan -LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2163,12 +2163,12 @@ WHERE EXISTS ( ) ---- logical_plan -LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Filter: join_t2.t2_int < UInt32(3) ---------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Filter: join_t2.t2_int < UInt32(3) +06)--------TableScan: join_t2 projection=[t2_id, t2_int] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2211,11 +2211,11 @@ WHERE EXISTS ( ) ---- logical_plan -LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---Filter: join_t1.t1_int < UInt32(3) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--Filter: join_t1.t1_int < UInt32(3) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--SubqueryAlias: __correlated_sq_1 +05)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2258,10 +2258,10 @@ WHERE NOT EXISTS ( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----TableScan: join_t2 projection=[t2_id] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2304,13 +2304,13 @@ WHERE NOT EXISTS ( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2353,13 +2353,13 @@ WHERE NOT EXISTS( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2404,13 +2404,13 @@ WHERE NOT EXISTS( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] query ITI SELECT * FROM join_t1 @@ -2435,13 +2435,13 @@ WHERE NOT EXISTS( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] query ITI SELECT * FROM join_t1 @@ -2539,22 +2539,22 @@ query TT EXPLAIN SELECT * FROM test_timestamps_tz_table as t1 JOIN test_timestamps_tz_table as t2 ON t1.millis = t2.millis ---- logical_plan -Inner Join: t1.millis = t2.millis ---SubqueryAlias: t1 -----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] ---SubqueryAlias: t2 -----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] +01)Inner Join: t1.millis = t2.millis +02)--SubqueryAlias: t1 +03)----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] +04)--SubqueryAlias: t2 +05)----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2613,11 +2613,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 join hashjoin_datatype_table_t2 t2 on t1.c1 = t2.c1 ---- logical_plan -Inner Join: t1.c1 = t2.c1 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Inner Join: t1.c1 = t2.c1 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] # hash_join_with_date32 query DDR?DDR? rowsort @@ -2632,11 +2632,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 left join hashjoin_datatype_table_t2 t2 on t1.c2 = t2.c2 ---- logical_plan -Left Join: t1.c2 = t2.c2 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Left Join: t1.c2 = t2.c2 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] # hash_join_with_date64 query DDR?DDR? rowsort @@ -2653,11 +2653,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 right join hashjoin_datatype_table_t1 t2 on t1.c3 = t2.c3 ---- logical_plan -Right Join: t1.c3 = t2.c3 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +01)Right Join: t1.c3 = t2.c3 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] # hash_join_with_decimal query DDR?DDR? rowsort @@ -2673,11 +2673,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 join hashjoin_datatype_table_t1 t2 on t1.c4 = t2.c4 ---- logical_plan -Inner Join: t1.c4 = t2.c4 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +01)Inner Join: t1.c4 = t2.c4 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] # hash_join_with_dictionary query DDR?DDR? rowsort @@ -2712,23 +2712,23 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 join hashjoin_datatype_table_t2 t2 on t1.c1 = t2.c1 ---- logical_plan -Inner Join: t1.c1 = t2.c1 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Inner Join: t1.c1 = t2.c1 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] physical_plan -SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] ---SortExec: expr=[c1@0 ASC] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] ---SortExec: expr=[c1@0 ASC] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] +02)--SortExec: expr=[c1@0 ASC] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)--SortExec: expr=[c1@0 ASC] +08)----CoalesceBatchesExec: target_batch_size=2 +09)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDR?DDR? rowsort @@ -2742,25 +2742,25 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 right join hashjoin_datatype_table_t2 t2 on t1.c3 = t2.c3 ---- logical_plan -Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] physical_plan -ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4] ---SortMergeJoin: join_type=Right, on=[(CAST(t1.c3 AS Decimal128(10, 2))@4, c3@2)] -----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 -----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -----SortExec: expr=[c3@2 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4] +02)--SortMergeJoin: join_type=Right, on=[(CAST(t1.c3 AS Decimal128(10, 2))@4, c3@2)] +03)----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 +06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)----SortExec: expr=[c3@2 ASC] +10)------CoalesceBatchesExec: target_batch_size=2 +11)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 +12)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDR?DDR? rowsort @@ -2811,18 +2811,18 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2852,18 +2852,18 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -2914,13 +2914,13 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2950,13 +2950,13 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -3008,18 +3008,18 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3030,18 +3030,18 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3090,13 +3090,13 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3107,13 +3107,13 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3176,29 +3176,29 @@ EXPLAIN SELECT * ORDER BY l_table.rn1 ---- logical_plan -Sort: l_table.rn1 ASC NULLS LAST ---Inner Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: l_table.rn1 ASC NULLS LAST +02)--Inner Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +06)----------TableScan: annotated_data projection=[a0, a, b, c, d] +07)----SubqueryAlias: r_table +08)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] ---SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] +02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +03)----SortExec: expr=[rn1@5 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +08)--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +10)----SortExec: expr=[a@1 ASC] +11)------CoalesceBatchesExec: target_batch_size=2 +12)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +13)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # sort merge join should propagate ordering equivalence of the right side # for right join. Hence final requirement rn1 ASC is already satisfied at @@ -3212,29 +3212,29 @@ EXPLAIN SELECT * ORDER BY r_table.rn1 ---- logical_plan -Sort: r_table.rn1 ASC NULLS LAST ---Right Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: r_table.rn1 ASC NULLS LAST +02)--Right Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------TableScan: annotated_data projection=[a0, a, b, c, d] +05)----SubqueryAlias: r_table +06)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +07)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] ---SortMergeJoin: join_type=Right, on=[(a@1, a@1)] -----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] +02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] +03)----SortExec: expr=[a@1 ASC] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +08)----SortExec: expr=[rn1@5 ASC NULLS LAST] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +13)--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +14)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering @@ -3250,34 +3250,34 @@ EXPLAIN SELECT * ORDER BY l_table.a ASC NULLS FIRST, l_table.b, l_table.c, r_table.rn1 ---- logical_plan -Sort: l_table.a ASC NULLS FIRST, l_table.b ASC NULLS LAST, l_table.c ASC NULLS LAST, r_table.rn1 ASC NULLS LAST ---Inner Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: l_table.a ASC NULLS FIRST, l_table.b ASC NULLS LAST, l_table.c ASC NULLS LAST, r_table.rn1 ASC NULLS LAST +02)--Inner Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +06)----------TableScan: annotated_data projection=[a0, a, b, c, d] +07)----SubqueryAlias: r_table +08)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +09)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +10)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] ---SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] -----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] +02)--SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] +03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +04)------SortExec: expr=[a@1 ASC] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +09)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +11)------SortExec: expr=[a@1 ASC] +12)--------CoalesceBatchesExec: target_batch_size=2 +13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +16)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +17)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3298,21 +3298,21 @@ EXPLAIN SELECT * ORDER BY r_table.rn1 ---- logical_plan -Sort: r_table.rn1 ASC NULLS LAST ---Inner Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: r_table.rn1 ASC NULLS LAST +02)--Inner Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------TableScan: annotated_data projection=[a0, a, b, c, d] +05)----SubqueryAlias: r_table +06)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +07)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +05)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # hash join should propagate ordering equivalence of the right side for RIGHT ANTI join. # Hence final requirement rn1 ASC is already satisfied at the end of HashJoinExec. @@ -3325,21 +3325,21 @@ EXPLAIN SELECT * ORDER BY r_table.rn1 ---- logical_plan -Sort: r_table.rn1 ASC NULLS LAST ---RightAnti Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------TableScan: annotated_data projection=[a] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: r_table.rn1 ASC NULLS LAST +02)--RightAnti Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------TableScan: annotated_data projection=[a] +05)----SubqueryAlias: r_table +06)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +07)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], has_header=true -----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], has_header=true +04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +05)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3350,21 +3350,21 @@ GROUP BY l.a, l.b, l.c ORDER BY l.a ASC NULLS FIRST; ---- logical_plan -Sort: l.a ASC NULLS FIRST ---Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 -----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] -------Inner Join: l.a = r.a ---------SubqueryAlias: l -----------TableScan: annotated_data projection=[a, b, c] ---------SubqueryAlias: r -----------TableScan: annotated_data projection=[a, b] +01)Sort: l.a ASC NULLS FIRST +02)--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +03)----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +04)------Inner Join: l.a = r.a +05)--------SubqueryAlias: l +06)----------TableScan: annotated_data projection=[a, b, c] +07)--------SubqueryAlias: r +08)----------TableScan: annotated_data projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3394,27 +3394,27 @@ GROUP BY row_n ORDER BY row_n ---- logical_plan -Projection: amount_usd ---Sort: row_n ASC NULLS LAST -----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n -------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] ---------Projection: l.a, l.d, row_n -----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) -------------SubqueryAlias: l ---------------TableScan: multiple_ordered_table projection=[a, d] -------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n ---------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------------SubqueryAlias: r -------------------TableScan: multiple_ordered_table projection=[a, d] +01)Projection: amount_usd +02)--Sort: row_n ASC NULLS LAST +03)----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +04)------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +05)--------Projection: l.a, l.d, row_n +06)----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +07)------------SubqueryAlias: l +08)--------------TableScan: multiple_ordered_table projection=[a, d] +09)------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +10)--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +11)----------------SubqueryAlias: r +12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] ---AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true ---------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # run query above in multiple partitions statement ok @@ -3433,32 +3433,32 @@ GROUP BY l.a, l.b, l.c ORDER BY l.a ASC NULLS FIRST; ---- logical_plan -Sort: l.a ASC NULLS FIRST ---Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 -----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] -------Inner Join: l.a = r.a ---------SubqueryAlias: l -----------TableScan: annotated_data projection=[a, b, c] ---------SubqueryAlias: r -----------TableScan: annotated_data projection=[a, b] +01)Sort: l.a ASC NULLS FIRST +02)--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +03)----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +04)------Inner Join: l.a = r.a +05)--------SubqueryAlias: l +06)----------TableScan: annotated_data projection=[a, b, c] +07)--------SubqueryAlias: r +08)----------TableScan: annotated_data projection=[a, b] physical_plan -SortPreservingMergeExec: [a@0 ASC] ---SortExec: expr=[a@0 ASC] -----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] ---------------CoalesceBatchesExec: target_batch_size=2 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@0 ASC] +02)--SortExec: expr=[a@0 ASC] +03)----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 +07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +10)------------------CoalesceBatchesExec: target_batch_size=2 +11)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +14)------------------CoalesceBatchesExec: target_batch_size=2 +15)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT * @@ -3466,16 +3466,16 @@ FROM annotated_data as l, annotated_data as r WHERE l.a > r.a ---- logical_plan -Inner Join: Filter: r.a < l.a ---SubqueryAlias: l -----TableScan: annotated_data projection=[a0, a, b, c, d] ---SubqueryAlias: r -----TableScan: annotated_data projection=[a0, a, b, c, d] +01)Inner Join: Filter: r.a < l.a +02)--SubqueryAlias: l +03)----TableScan: annotated_data projection=[a0, a, b, c, d] +04)--SubqueryAlias: r +05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 ---RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +04)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # Currently datafusion cannot pushdown filter conditions with scalar UDF into # cross join. @@ -3485,19 +3485,19 @@ FROM annotated_data as t1, annotated_data as t2 WHERE EXAMPLE(t1.a, t2.a) > 3 ---- logical_plan -Filter: example(CAST(t1.a AS Float64), CAST(t2.a AS Float64)) > Float64(3) ---CrossJoin: -----SubqueryAlias: t1 -------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: t2 -------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Filter: example(CAST(t1.a AS Float64), CAST(t2.a AS Float64)) > Float64(3) +02)--CrossJoin: +03)----SubqueryAlias: t1 +04)------TableScan: annotated_data projection=[a0, a, b, c, d] +05)----SubqueryAlias: t2 +06)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---FilterExec: example(CAST(a@1 AS Float64), CAST(a@6 AS Float64)) > 3 -----CrossJoinExec -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)CoalesceBatchesExec: target_batch_size=2 +02)--FilterExec: example(CAST(a@1 AS Float64), CAST(a@6 AS Float64)) > 3 +03)----CrossJoinExec +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true #### # Config teardown diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 7508b44f02d4..5d3c23d5130b 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -49,14 +49,14 @@ query TT EXPLAIN SELECT count(*) from json_test ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---TableScan: json_test projection=[] +01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--TableScan: json_test projection=[] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]} +01)AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]} query ? SELECT mycol FROM single_nan diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index f70e4cb30e6b..b4138f38ea2b 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -307,12 +307,12 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---Limit: skip=11, fetch=3 -----TableScan: t1 projection=[], fetch=14 +01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Limit: skip=11, fetch=3 +03)----TableScan: t1 projection=[], fetch=14 physical_plan -ProjectionExec: expr=[0 as COUNT(*)] ---PlaceholderRowExec +01)ProjectionExec: expr=[0 as COUNT(*)] +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); @@ -325,12 +325,12 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---Limit: skip=8, fetch=3 -----TableScan: t1 projection=[], fetch=11 +01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Limit: skip=8, fetch=3 +03)----TableScan: t1 projection=[], fetch=11 physical_plan -ProjectionExec: expr=[2 as COUNT(*)] ---PlaceholderRowExec +01)ProjectionExec: expr=[2 as COUNT(*)] +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -343,12 +343,12 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 OFFSET 8); ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---Limit: skip=8, fetch=None -----TableScan: t1 projection=[] +01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Limit: skip=8, fetch=None +03)----TableScan: t1 projection=[] physical_plan -ProjectionExec: expr=[2 as COUNT(*)] ---PlaceholderRowExec +01)ProjectionExec: expr=[2 as COUNT(*)] +02)--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -360,21 +360,21 @@ query TT EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---Projection: -----Limit: skip=6, fetch=3 -------Filter: t1.a > Int32(3) ---------TableScan: t1 projection=[a] +01)Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Projection: +03)----Limit: skip=6, fetch=3 +04)------Filter: t1.a > Int32(3) +05)--------TableScan: t1 projection=[a] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------ProjectionExec: expr=[] -----------GlobalLimitExec: skip=6, fetch=3 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------FilterExec: a@0 > 3 -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------ProjectionExec: expr=[] +06)----------GlobalLimitExec: skip=6, fetch=3 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------FilterExec: a@0 > 3 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); @@ -396,14 +396,14 @@ query TT EXPLAIN SELECT DISTINCT i FROM t1000; ---- logical_plan -Aggregate: groupBy=[[t1000.i]], aggr=[[]] ---TableScan: t1000 projection=[i] +01)Aggregate: groupBy=[[t1000.i]], aggr=[[]] +02)--TableScan: t1000 projection=[i] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] ---CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 -------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] ---------MemoryExec: partitions=4 +01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] +05)--------MemoryExec: partitions=4 statement ok set datafusion.explain.show_sizes = true; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 7863bf445499..415fabf224d7 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -54,12 +54,12 @@ query TT explain select * from table_with_map where int_field > 0; ---- logical_plan -Filter: table_with_map.int_field > Int64(0) ---TableScan: table_with_map projection=[int_field, map_field] +01)Filter: table_with_map.int_field > Int64(0) +02)--TableScan: table_with_map projection=[int_field, map_field] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: int_field@0 > 0 -----MemoryExec: partitions=1, partition_sizes=[0] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: int_field@0 > 0 +03)----MemoryExec: partitions=1, partition_sizes=[0] statement ok drop table table_with_map; diff --git a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt index 48e79621d850..3f12dfcda88a 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -40,14 +40,14 @@ FROM multiple_ordered_table ORDER BY a_big ASC, b ASC; ---- logical_plan -Sort: a_big ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Projection: CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: a_big ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Projection: CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -SortPreservingMergeExec: [a_big@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ---ProjectionExec: expr=[CAST(a@0 AS Int64) as a_big, b@1 as b] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +02)--ProjectionExec: expr=[CAST(a@0 AS Int64) as a_big, b@1 as b] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true query TT EXPLAIN @@ -56,14 +56,14 @@ FROM multiple_ordered_table ORDER BY a ASC, b ASC; ---- logical_plan -Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@2 ASC NULLS LAST] ---ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@2 ASC NULLS LAST] +02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # Cast to larger types as well as preserving ordering # doesn't invalidate lexicographical ordering. @@ -77,14 +77,14 @@ FROM multiple_ordered_table ORDER BY a_big ASC, b ASC; ---- logical_plan -Sort: a_big ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: a_big ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -SortPreservingMergeExec: [a_big@1 ASC NULLS LAST,b@2 ASC NULLS LAST] ---ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST,b@2 ASC NULLS LAST] +02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # test for common rename query TT @@ -94,12 +94,12 @@ FROM multiple_ordered_table ORDER BY a_big ASC, b ASC; ---- logical_plan -Sort: a_big ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Projection: multiple_ordered_table.a, multiple_ordered_table.a AS a_big, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: a_big ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Projection: multiple_ordered_table.a, multiple_ordered_table.a AS a_big, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true query TT EXPLAIN @@ -108,12 +108,12 @@ FROM multiple_ordered_table ORDER BY a ASC, b ASC; ---- logical_plan -Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Projection: multiple_ordered_table.a, multiple_ordered_table.a AS a_big, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Projection: multiple_ordered_table.a, multiple_ordered_table.a AS a_big, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # test for cast Utf8 @@ -128,15 +128,15 @@ FROM multiple_ordered_table ORDER BY a_str ASC, b ASC; ---- logical_plan -Sort: a_str ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Projection: CAST(multiple_ordered_table.a AS Utf8) AS a_str, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: a_str ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Projection: CAST(multiple_ordered_table.a AS Utf8) AS a_str, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -SortPreservingMergeExec: [a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ---SortExec: expr=[a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST] -----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +02)--SortExec: expr=[a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # We cannot determine a+b is ordered from the # invariant [a ASC, b ASC] is satisfied. Hence @@ -148,11 +148,11 @@ FROM multiple_ordered_table ORDER BY a + b ASC; ---- logical_plan -Sort: multiple_ordered_table.a + multiple_ordered_table.b ASC NULLS LAST ---TableScan: multiple_ordered_table projection=[a, b] +01)Sort: multiple_ordered_table.a + multiple_ordered_table.b ASC NULLS LAST +02)--TableScan: multiple_ordered_table projection=[a, b] physical_plan -SortExec: expr=[a@0 + b@1 ASC NULLS LAST] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortExec: expr=[a@0 + b@1 ASC NULLS LAST] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # With similar reasoning above. It is not guaranteed sum_expr is ordered # Hence we should see a SortExec with sum_expr ASC in the plan. @@ -163,12 +163,12 @@ FROM multiple_ordered_table ORDER BY sum_expr ASC; ---- logical_plan -Sort: sum_expr ASC NULLS LAST ---Projection: CAST(multiple_ordered_table.a + multiple_ordered_table.b AS Int64) AS sum_expr, multiple_ordered_table.a, multiple_ordered_table.b -----TableScan: multiple_ordered_table projection=[a, b] +01)Sort: sum_expr ASC NULLS LAST +02)--Projection: CAST(multiple_ordered_table.a + multiple_ordered_table.b AS Int64) AS sum_expr, multiple_ordered_table.a, multiple_ordered_table.b +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -SortPreservingMergeExec: [sum_expr@0 ASC NULLS LAST] ---SortExec: expr=[sum_expr@0 ASC NULLS LAST] -----ProjectionExec: expr=[CAST(a@0 + b@1 AS Int64) as sum_expr, a@0 as a, b@1 as b] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [sum_expr@0 ASC NULLS LAST] +02)--SortExec: expr=[sum_expr@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[CAST(a@0 + b@1 AS Int64) as sum_expr, a@0 as a, b@1 as b] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 9366a9b3b3c8..ba9eedcbbd34 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -28,12 +28,12 @@ query TT explain SELECT * FROM a WHERE c0 < 1; ---- logical_plan -Filter: a.c0 < Int32(1) ---TableScan: a projection=[c0] +01)Filter: a.c0 < Int32(1) +02)--TableScan: a projection=[c0] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: c0@0 < 1 -----MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: c0@0 < 1 +03)----MemoryExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -47,11 +47,11 @@ query TT explain SELECT * FROM a WHERE c0 < 1; ---- logical_plan -Filter: a.c0 < Int32(1) ---TableScan: a projection=[c0] +01)Filter: a.c0 < Int32(1) +02)--TableScan: a projection=[c0] physical_plan -FilterExec: c0@0 < 1 ---MemoryExec: partitions=1, partition_sizes=[1] +01)FilterExec: c0@0 < 1 +02)--MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.execution.coalesce_batches = true @@ -69,12 +69,12 @@ query TT explain SELECT * FROM a WHERE c0 < 1; ---- logical_plan -Filter: a.c0 < Int32(1) ---TableScan: a projection=[c0] +01)Filter: a.c0 < Int32(1) +02)--TableScan: a projection=[c0] physical_plan -CoalesceBatchesExec: target_batch_size=1234 ---FilterExec: c0@0 < 1 -----MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=1234 +02)--FilterExec: c0@0 < 1 +03)----MemoryExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index f63179a369c5..1ad74398b911 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -159,13 +159,13 @@ query TT explain SELECT c1, c2 FROM aggregate_test_100 ORDER BY c2, c3, c2 ---- logical_plan -Projection: aggregate_test_100.c1, aggregate_test_100.c2 ---Sort: aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST -----TableScan: aggregate_test_100 projection=[c1, c2, c3] +01)Projection: aggregate_test_100.c1, aggregate_test_100.c2 +02)--Sort: aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST +03)----TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] ---SortExec: expr=[c2@1 ASC NULLS LAST,c3@2 ASC NULLS LAST] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] +02)--SortExec: expr=[c2@1 ASC NULLS LAST,c3@2 ASC NULLS LAST] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true query II SELECT c2, c3 FROM aggregate_test_100 ORDER BY c2, c3, c2 @@ -434,14 +434,14 @@ FROM multiple_ordered_table ORDER BY result; ---- logical_plan -Sort: result ASC NULLS LAST ---Projection: multiple_ordered_table.b + multiple_ordered_table.a + multiple_ordered_table.c AS result -----TableScan: multiple_ordered_table projection=[a, b, c] +01)Sort: result ASC NULLS LAST +02)--Projection: multiple_ordered_table.b + multiple_ordered_table.a + multiple_ordered_table.c AS result +03)----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan -SortPreservingMergeExec: [result@0 ASC NULLS LAST] ---ProjectionExec: expr=[b@1 + a@0 + c@2 as result] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST], [b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)SortPreservingMergeExec: [result@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[b@1 + a@0 + c@2 as result] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST], [b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok drop table multiple_ordered_table; @@ -464,14 +464,14 @@ FROM csv_with_timestamps ORDER BY db15; ---- logical_plan -Sort: db15 ASC NULLS LAST ---Projection: date_bin(IntervalMonthDayNano("900000000000"), csv_with_timestamps.ts, TimestampNanosecond(1659537600000000000, None)) AS db15 -----TableScan: csv_with_timestamps projection=[ts] +01)Sort: db15 ASC NULLS LAST +02)--Projection: date_bin(IntervalMonthDayNano("900000000000"), csv_with_timestamps.ts, TimestampNanosecond(1659537600000000000, None)) AS db15 +03)----TableScan: csv_with_timestamps projection=[ts] physical_plan -SortPreservingMergeExec: [db15@0 ASC NULLS LAST] ---ProjectionExec: expr=[date_bin(900000000000, ts@0, 1659537600000000000) as db15] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false +01)SortPreservingMergeExec: [db15@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[date_bin(900000000000, ts@0, 1659537600000000000) as db15] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false query TT EXPLAIN SELECT DATE_TRUNC('DAY', ts) as dt_day @@ -479,14 +479,14 @@ FROM csv_with_timestamps ORDER BY dt_day; ---- logical_plan -Sort: dt_day ASC NULLS LAST ---Projection: date_trunc(Utf8("DAY"), csv_with_timestamps.ts) AS dt_day -----TableScan: csv_with_timestamps projection=[ts] +01)Sort: dt_day ASC NULLS LAST +02)--Projection: date_trunc(Utf8("DAY"), csv_with_timestamps.ts) AS dt_day +03)----TableScan: csv_with_timestamps projection=[ts] physical_plan -SortPreservingMergeExec: [dt_day@0 ASC NULLS LAST] ---ProjectionExec: expr=[date_trunc(DAY, ts@0) as dt_day] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false +01)SortPreservingMergeExec: [dt_day@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[date_trunc(DAY, ts@0) as dt_day] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false statement ok drop table csv_with_timestamps; @@ -522,14 +522,14 @@ FROM aggregate_test_100 ORDER BY atan_c11; ---- logical_plan -Sort: atan_c11 ASC NULLS LAST ---Projection: atan(aggregate_test_100.c11) AS atan_c11 -----TableScan: aggregate_test_100 projection=[c11] +01)Sort: atan_c11 ASC NULLS LAST +02)--Projection: atan(aggregate_test_100.c11) AS atan_c11 +03)----TableScan: aggregate_test_100 projection=[c11] physical_plan -SortPreservingMergeExec: [atan_c11@0 ASC NULLS LAST] ---ProjectionExec: expr=[atan(c11@0) as atan_c11] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [atan_c11@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[atan(c11@0) as atan_c11] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT CEIL(c11) as ceil_c11 @@ -537,14 +537,14 @@ FROM aggregate_test_100 ORDER BY ceil_c11; ---- logical_plan -Sort: ceil_c11 ASC NULLS LAST ---Projection: ceil(aggregate_test_100.c11) AS ceil_c11 -----TableScan: aggregate_test_100 projection=[c11] +01)Sort: ceil_c11 ASC NULLS LAST +02)--Projection: ceil(aggregate_test_100.c11) AS ceil_c11 +03)----TableScan: aggregate_test_100 projection=[c11] physical_plan -SortPreservingMergeExec: [ceil_c11@0 ASC NULLS LAST] ---ProjectionExec: expr=[ceil(c11@0) as ceil_c11] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [ceil_c11@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[ceil(c11@0) as ceil_c11] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT LOG(c11, c12) as log_c11_base_c12 @@ -552,14 +552,14 @@ query TT ORDER BY log_c11_base_c12; ---- logical_plan -Sort: log_c11_base_c12 ASC NULLS LAST ---Projection: log(CAST(aggregate_test_100.c11 AS Float64), aggregate_test_100.c12) AS log_c11_base_c12 -----TableScan: aggregate_test_100 projection=[c11, c12] +01)Sort: log_c11_base_c12 ASC NULLS LAST +02)--Projection: log(CAST(aggregate_test_100.c11 AS Float64), aggregate_test_100.c12) AS log_c11_base_c12 +03)----TableScan: aggregate_test_100 projection=[c11, c12] physical_plan -SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] ---ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c11_base_c12] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC]], has_header=true +01)SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c11_base_c12] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC]], has_header=true query TT EXPLAIN SELECT LOG(c12, c11) as log_c12_base_c11 @@ -567,14 +567,14 @@ FROM aggregate_test_100 ORDER BY log_c12_base_c11 DESC; ---- logical_plan -Sort: log_c12_base_c11 DESC NULLS FIRST ---Projection: log(aggregate_test_100.c12, CAST(aggregate_test_100.c11 AS Float64)) AS log_c12_base_c11 -----TableScan: aggregate_test_100 projection=[c11, c12] +01)Sort: log_c12_base_c11 DESC NULLS FIRST +02)--Projection: log(aggregate_test_100.c12, CAST(aggregate_test_100.c11 AS Float64)) AS log_c12_base_c11 +03)----TableScan: aggregate_test_100 projection=[c11, c12] physical_plan -SortPreservingMergeExec: [log_c12_base_c11@0 DESC] ---ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c12_base_c11] -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC]], has_header=true +01)SortPreservingMergeExec: [log_c12_base_c11@0 DESC] +02)--ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c12_base_c11] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC]], has_header=true statement ok drop table aggregate_test_100; @@ -633,8 +633,8 @@ query TT EXPLAIN SELECT o_orderkey, o_orderstatus FROM orders ORDER BY o_orderkey ASC ---- logical_plan -Sort: orders.o_orderkey ASC NULLS LAST ---TableScan: orders projection=[o_orderkey, o_orderstatus] +01)Sort: orders.o_orderkey ASC NULLS LAST +02)--TableScan: orders projection=[o_orderkey, o_orderstatus] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/orders.csv]]}, projection=[o_orderkey, o_orderstatus], output_ordering=[o_orderkey@0 ASC NULLS LAST], has_header=true @@ -677,11 +677,11 @@ query TT EXPLAIN SELECT * FROM t1 ORDER BY id DESC, id, name, id ASC; ---- logical_plan -Sort: t1.id DESC NULLS FIRST, t1.name ASC NULLS LAST ---TableScan: t1 projection=[id, name] +01)Sort: t1.id DESC NULLS FIRST, t1.name ASC NULLS LAST +02)--TableScan: t1 projection=[id, name] physical_plan -SortExec: expr=[id@0 DESC,name@1 ASC NULLS LAST] ---MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[id@0 DESC,name@1 ASC NULLS LAST] +02)--MemoryExec: partitions=1, partition_sizes=[1] query IT SELECT * FROM t1 ORDER BY id ASC, id, name, id DESC; @@ -696,11 +696,11 @@ query TT EXPLAIN SELECT * FROM t1 ORDER BY id ASC, id, name, id DESC; ---- logical_plan -Sort: t1.id ASC NULLS LAST, t1.name ASC NULLS LAST ---TableScan: t1 projection=[id, name] +01)Sort: t1.id ASC NULLS LAST, t1.name ASC NULLS LAST +02)--TableScan: t1 projection=[id, name] physical_plan -SortExec: expr=[id@0 ASC NULLS LAST,name@1 ASC NULLS LAST] ---MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[id@0 ASC NULLS LAST,name@1 ASC NULLS LAST] +02)--MemoryExec: partitions=1, partition_sizes=[1] # Minimal reproduction of issue 5970 @@ -750,40 +750,40 @@ SELECT * FROM v ORDER BY 1, 2; ---- logical_plan -Sort: u.m ASC NULLS LAST, u.t ASC NULLS LAST ---Union -----SubqueryAlias: u -------Projection: Int64(0) AS m, m0.t ---------Aggregate: groupBy=[[Int64(0), m0.t]], aggr=[[]] -----------SubqueryAlias: m0 -------------Projection: column1 AS t ---------------Values: (Int64(0)), (Int64(1)), (Int64(2)) -----SubqueryAlias: v -------Projection: Int64(1) AS m, m1.t ---------Aggregate: groupBy=[[Int64(1), m1.t]], aggr=[[]] -----------SubqueryAlias: m1 -------------Projection: column1 AS t ---------------Values: (Int64(0)), (Int64(1)) +01)Sort: u.m ASC NULLS LAST, u.t ASC NULLS LAST +02)--Union +03)----SubqueryAlias: u +04)------Projection: Int64(0) AS m, m0.t +05)--------Aggregate: groupBy=[[Int64(0), m0.t]], aggr=[[]] +06)----------SubqueryAlias: m0 +07)------------Projection: column1 AS t +08)--------------Values: (Int64(0)), (Int64(1)), (Int64(2)) +09)----SubqueryAlias: v +10)------Projection: Int64(1) AS m, m1.t +11)--------Aggregate: groupBy=[[Int64(1), m1.t]], aggr=[[]] +12)----------SubqueryAlias: m1 +13)------------Projection: column1 AS t +14)--------------Values: (Int64(0)), (Int64(1)) physical_plan -SortPreservingMergeExec: [m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] ---SortExec: expr=[m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] -----InterleaveExec -------ProjectionExec: expr=[Int64(0)@0 as m, t@1 as t] ---------AggregateExec: mode=FinalPartitioned, gby=[Int64(0)@0 as Int64(0), t@1 as t], aggr=[], ordering_mode=PartiallySorted([0]) -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([Int64(0)@0, t@1], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[], ordering_mode=PartiallySorted([0]) -------------------ProjectionExec: expr=[column1@0 as t] ---------------------ValuesExec -------ProjectionExec: expr=[Int64(1)@0 as m, t@1 as t] ---------AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1), t@1 as t], aggr=[], ordering_mode=PartiallySorted([0]) -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([Int64(1)@0, t@1], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[], ordering_mode=PartiallySorted([0]) -------------------ProjectionExec: expr=[column1@0 as t] ---------------------ValuesExec +01)SortPreservingMergeExec: [m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] +02)--SortExec: expr=[m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] +03)----InterleaveExec +04)------ProjectionExec: expr=[Int64(0)@0 as m, t@1 as t] +05)--------AggregateExec: mode=FinalPartitioned, gby=[Int64(0)@0 as Int64(0), t@1 as t], aggr=[], ordering_mode=PartiallySorted([0]) +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([Int64(0)@0, t@1], 2), input_partitions=2 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[], ordering_mode=PartiallySorted([0]) +10)------------------ProjectionExec: expr=[column1@0 as t] +11)--------------------ValuesExec +12)------ProjectionExec: expr=[Int64(1)@0 as m, t@1 as t] +13)--------AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1), t@1 as t], aggr=[], ordering_mode=PartiallySorted([0]) +14)----------CoalesceBatchesExec: target_batch_size=8192 +15)------------RepartitionExec: partitioning=Hash([Int64(1)@0, t@1], 2), input_partitions=2 +16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)----------------AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[], ordering_mode=PartiallySorted([0]) +18)------------------ProjectionExec: expr=[column1@0 as t] +19)--------------------ValuesExec ##### # Multi column sorting with lists diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 3cc52666d533..1b25406d8172 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -87,12 +87,12 @@ FROM test_table ORDER BY string_col, int_col; ---- logical_plan -Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST ---TableScan: test_table projection=[int_col, string_col] +01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST +02)--TableScan: test_table projection=[int_col, string_col] physical_plan -SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] ---SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] -----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col] # Tear down test_table: statement ok @@ -118,11 +118,11 @@ FROM test_table ORDER BY string_col, int_col; ---- logical_plan -Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST ---TableScan: test_table projection=[int_col, string_col] +01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST +02)--TableScan: test_table projection=[int_col, string_col] physical_plan -SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] ---ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +02)--ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] # Add another file to the directory underlying test_table query ITID @@ -140,12 +140,12 @@ FROM test_table ORDER BY string_col, int_col; ---- logical_plan -Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST ---TableScan: test_table projection=[int_col, string_col] +01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST +02)--TableScan: test_table projection=[int_col, string_col] physical_plan -SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] ---SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] -----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col] # Perform queries using MIN and MAX diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 33c9ff7c3eed..22af0cf894d1 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -377,40 +377,40 @@ query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR column1 = 'bar' OR column1 = 'fazzz' ---- logical_plan -Filter: test.column1 = Utf8("foo") OR test.column1 = Utf8("bar") OR test.column1 = Utf8("fazzz") ---TableScan: test projection=[column1] +01)Filter: test.column1 = Utf8("foo") OR test.column1 = Utf8("bar") OR test.column1 = Utf8("fazzz") +02)--TableScan: test projection=[column1] # Number of OR statements is greater than threshold query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR column1 = 'bar' OR column1 = 'fazzz' OR column1 = 'barfoo' ---- logical_plan -Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) ---TableScan: test projection=[column1] +01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) +02)--TableScan: test projection=[column1] # Complex OR statements query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR column1 = 'bar' OR column1 = 'fazzz' OR column1 = 'barfoo' OR false OR column1 = 'foobar' ---- logical_plan -Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo"), Utf8("foobar")]) ---TableScan: test projection=[column1] +01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo"), Utf8("foobar")]) +02)--TableScan: test projection=[column1] # Balanced OR structures query TT EXPLAIN SELECT * FROM test WHERE (column1 = 'foo' OR column1 = 'bar') OR (column1 = 'fazzz' OR column1 = 'barfoo') ---- logical_plan -Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) ---TableScan: test projection=[column1] +01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) +02)--TableScan: test projection=[column1] # Right-deep OR structures query TT EXPLAIN SELECT * FROM test WHERE column1 = 'foo' OR (column1 = 'bar' OR (column1 = 'fazzz' OR column1 = 'barfoo')) ---- logical_plan -Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) ---TableScan: test projection=[column1] +01)Filter: test.column1 IN ([Utf8("foo"), Utf8("bar"), Utf8("fazzz"), Utf8("barfoo")]) +02)--TableScan: test projection=[column1] # Not simplifiable, mixed column query TT @@ -418,8 +418,8 @@ EXPLAIN SELECT * FROM aggregate_test_100 WHERE (c2 = 1 OR c3 = 100) OR (c2 = 2 OR c2 = 3 OR c2 = 4) ---- logical_plan -Filter: aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4) ---TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4)] +01)Filter: aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4) +02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 = Int8(2) OR aggregate_test_100.c2 = Int8(3) OR aggregate_test_100.c2 = Int8(4)] # Partially simplifiable, mixed column query TT @@ -427,8 +427,8 @@ EXPLAIN SELECT * FROM aggregate_test_100 WHERE (c2 = 1 OR c3 = 100) OR (c2 = 2 OR c2 = 3 OR c2 = 4 OR c2 = 5) ---- logical_plan -Filter: aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)]) ---TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)])] +01)Filter: aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)]) +02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c2 = Int8(1) OR aggregate_test_100.c3 = Int16(100) OR aggregate_test_100.c2 IN ([Int8(2), Int8(3), Int8(4), Int8(5)])] statement ok set datafusion.explain.logical_plan_only = false @@ -628,27 +628,27 @@ OR ) ---- logical_plan -Projection: lineitem.l_partkey ---Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#12") AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) -----Filter: lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) -------TableScan: lineitem projection=[l_partkey, l_quantity], partial_filters=[lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] -----Filter: (part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) -------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] +01)Projection: lineitem.l_partkey +02)--Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#12") AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) +03)----Filter: lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) +04)------TableScan: lineitem projection=[l_partkey, l_quantity], partial_filters=[lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] +05)----Filter: (part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) +06)------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] -----CoalesceBatchesExec: target_batch_size=8192 -------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------CoalesceBatchesExec: target_batch_size=8192 -----------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true -----CoalesceBatchesExec: target_batch_size=8192 -------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------CoalesceBatchesExec: target_batch_size=8192 -----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true +01)CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true +09)----CoalesceBatchesExec: target_batch_size=8192 +10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +11)--------CoalesceBatchesExec: target_batch_size=8192 +12)----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -715,36 +715,36 @@ OR GROUP BY p_partkey; ---- logical_plan -Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)]] ---Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, partsupp.ps_suppkey -----Inner Join: part.p_partkey = partsupp.ps_partkey -------Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey ---------Inner Join: lineitem.l_partkey = part.p_partkey -----------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount] -----------Projection: part.p_partkey -------------Filter: part.p_brand = Utf8("Brand#12") OR part.p_brand = Utf8("Brand#23") ---------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8("Brand#12") OR part.p_brand = Utf8("Brand#23")] -------TableScan: partsupp projection=[ps_partkey, ps_suppkey] +01)Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)]] +02)--Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, partsupp.ps_suppkey +03)----Inner Join: part.p_partkey = partsupp.ps_partkey +04)------Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey +05)--------Inner Join: lineitem.l_partkey = part.p_partkey +06)----------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount] +07)----------Projection: part.p_partkey +08)------------Filter: part.p_brand = Utf8("Brand#12") OR part.p_brand = Utf8("Brand#23") +09)--------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8("Brand#12") OR part.p_brand = Utf8("Brand#23")] +10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan -AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] ---CoalesceBatchesExec: target_batch_size=8192 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] -------CoalesceBatchesExec: target_batch_size=8192 ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true +10)----------CoalesceBatchesExec: target_batch_size=8192 +11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +12)--------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 +15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +17)------CoalesceBatchesExec: target_batch_size=8192 +18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +19)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification @@ -767,12 +767,12 @@ query TT explain select x from t where x IN (1,2,3,4,5) AND x NOT IN (1,2,3,4); ---- logical_plan -Filter: t.x = Int32(5) ---TableScan: t projection=[x] +01)Filter: t.x = Int32(5) +02)--TableScan: t projection=[x] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: x@0 = 5 -----MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: x@0 = 5 +03)----MemoryExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x NOT IN (1,2,3,4,5) AND x IN (1,2,3); @@ -781,4 +781,4 @@ logical_plan EmptyRelation physical_plan EmptyExec statement ok -drop table t; \ No newline at end of file +drop table t; diff --git a/datafusion/sqllogictest/test_files/references.slt b/datafusion/sqllogictest/test_files/references.slt index c09addb3eec2..e276e322acab 100644 --- a/datafusion/sqllogictest/test_files/references.slt +++ b/datafusion/sqllogictest/test_files/references.slt @@ -101,13 +101,13 @@ query TT EXPLAIN (SELECT "....", "...." AS c3 FROM test ORDER BY "...."); ---- logical_plan -Sort: test..... ASC NULLS LAST ---Projection: test....., test..... AS c3 -----TableScan: test projection=[....] +01)Sort: test..... ASC NULLS LAST +02)--Projection: test....., test..... AS c3 +03)----TableScan: test projection=[....] physical_plan -SortExec: expr=[....@0 ASC NULLS LAST] ---ProjectionExec: expr=[....@0 as ...., ....@0 as c3] -----MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[....@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[....@0 as ...., ....@0 as c3] +03)----MemoryExec: partitions=1, partition_sizes=[1] # Partial qualified name diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 594c52f12d75..086270a7098f 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -40,15 +40,15 @@ query TT EXPLAIN SELECT column1, SUM(column2) FROM parquet_table GROUP BY column1; ---- logical_plan -Aggregate: groupBy=[[parquet_table.column1]], aggr=[[SUM(CAST(parquet_table.column2 AS Int64))]] ---TableScan: parquet_table projection=[column1, column2] +01)Aggregate: groupBy=[[parquet_table.column1]], aggr=[[SUM(CAST(parquet_table.column2 AS Int64))]] +02)--TableScan: parquet_table projection=[column1, column2] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] ---CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 -------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] # disable round robin repartitioning statement ok @@ -58,14 +58,14 @@ query TT EXPLAIN SELECT column1, SUM(column2) FROM parquet_table GROUP BY column1; ---- logical_plan -Aggregate: groupBy=[[parquet_table.column1]], aggr=[[SUM(CAST(parquet_table.column2 AS Int64))]] ---TableScan: parquet_table projection=[column1, column2] +01)Aggregate: groupBy=[[parquet_table.column1]], aggr=[[SUM(CAST(parquet_table.column2 AS Int64))]] +02)--TableScan: parquet_table projection=[column1, column2] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] ---CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 -------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] ---------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[SUM(parquet_table.column2)] +05)--------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] # Cleanup @@ -117,13 +117,13 @@ query TT EXPLAIN SELECT c1, c2, c3 FROM sink_table WHERE c3 > 0 LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Filter: sink_table.c3 > Int16(0) -----TableScan: sink_table projection=[c1, c2, c3] +01)Limit: skip=0, fetch=5 +02)--Filter: sink_table.c3 > Int16(0) +03)----TableScan: sink_table projection=[c1, c2, c3] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---CoalescePartitionsExec -----CoalesceBatchesExec: target_batch_size=8192 -------FilterExec: c3@2 > 0 ---------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--CoalescePartitionsExec +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: c3@2 > 0 +05)--------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index f9699a5fda8f..d41d36536ceb 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -56,12 +56,12 @@ query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- logical_plan -Filter: parquet_table.column1 != Int32(42) ---TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +01)Filter: parquet_table.column1 != Int32(42) +02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: column1@0 != 42 -----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: column1@0 != 42 +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -72,12 +72,12 @@ query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- logical_plan -Filter: parquet_table.column1 != Int32(42) ---TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +01)Filter: parquet_table.column1 != Int32(42) +02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: column1@0 != 42 -----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: column1@0 != 42 +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -94,15 +94,15 @@ query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42 ORDER BY column1; ---- logical_plan -Sort: parquet_table.column1 ASC NULLS LAST ---Filter: parquet_table.column1 != Int32(42) -----TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +01)Sort: parquet_table.column1 ASC NULLS LAST +02)--Filter: parquet_table.column1 != Int32(42) +03)----TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan -SortPreservingMergeExec: [column1@0 ASC NULLS LAST] ---SortExec: expr=[column1@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 -------FilterExec: column1@0 != 42 ---------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..205], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:205..405, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:5..210], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:210..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] +02)--SortExec: expr=[column1@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: column1@0 != 42 +05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..205], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:205..405, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:5..210], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:210..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -131,14 +131,14 @@ query TT EXPLAIN SELECT column1 FROM parquet_table_with_order WHERE column1 <> 42 ORDER BY column1; ---- logical_plan -Sort: parquet_table_with_order.column1 ASC NULLS LAST ---Filter: parquet_table_with_order.column1 != Int32(42) -----TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] +01)Sort: parquet_table_with_order.column1 ASC NULLS LAST +02)--Filter: parquet_table_with_order.column1 != Int32(42) +03)----TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] physical_plan -SortPreservingMergeExec: [column1@0 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: column1@0 != 42 -------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..207], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:207..414], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:202..405]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: column1@0 != 42 +04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..207], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:207..414], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:202..405]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # Cleanup statement ok @@ -180,12 +180,12 @@ query TT EXPLAIN SELECT column1 FROM csv_table WHERE column1 <> 42; ---- logical_plan -Filter: csv_table.column1 != Int32(42) ---TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] +01)Filter: csv_table.column1 != Int32(42) +02)--TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: column1@0 != 42 -----CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], has_header=true +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: column1@0 != 42 +03)----CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], has_header=true # Cleanup statement ok @@ -223,12 +223,12 @@ query TT EXPLAIN SELECT column1 FROM "json_table" WHERE column1 <> 42; ---- logical_plan -Filter: json_table.column1 != Int32(42) ---TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] +01)Filter: json_table.column1 != Int32(42) +02)--TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] physical_plan -CoalesceBatchesExec: target_batch_size=8192 ---FilterExec: column1@0 != 42 -----JsonExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: column1@0 != 42 +03)----JsonExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1] # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index e0c600c1484c..987fb399667c 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -2031,11 +2031,11 @@ query TT EXPLAIN SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; ---- logical_plan -Projection: simple_string.letter, simple_string.letter = Utf8("A") AS simple_string.letter = left(Utf8("APACHE"),Int64(1)) ---TableScan: simple_string projection=[letter] +01)Projection: simple_string.letter, simple_string.letter = Utf8("A") AS simple_string.letter = left(Utf8("APACHE"),Int64(1)) +02)--TableScan: simple_string projection=[letter] physical_plan -ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] +02)--MemoryExec: partitions=1, partition_sizes=[1] query TB SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; @@ -2050,11 +2050,11 @@ query TT EXPLAIN SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; ---- logical_plan -Projection: simple_string.letter, simple_string.letter = left(simple_string.letter2, Int64(1)) ---TableScan: simple_string projection=[letter, letter2] +01)Projection: simple_string.letter, simple_string.letter = left(simple_string.letter2, Int64(1)) +02)--TableScan: simple_string projection=[letter, letter2] physical_plan -ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] +02)--MemoryExec: partitions=1, partition_sizes=[1] query TB SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 8975acb3f537..fc6897849177 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -526,16 +526,16 @@ query TT EXPLAIN SELECT * FROM ((SELECT column1 FROM foo) "T1" CROSS JOIN (SELECT column2 FROM foo) "T2") AS "F" ---- logical_plan -SubqueryAlias: F ---CrossJoin: -----SubqueryAlias: T1 -------TableScan: foo projection=[column1] -----SubqueryAlias: T2 -------TableScan: foo projection=[column2] +01)SubqueryAlias: F +02)--CrossJoin: +03)----SubqueryAlias: T1 +04)------TableScan: foo projection=[column1] +05)----SubqueryAlias: T2 +06)------TableScan: foo projection=[column2] physical_plan -CrossJoinExec ---MemoryExec: partitions=1, partition_sizes=[1] ---MemoryExec: partitions=1, partition_sizes=[1] +01)CrossJoinExec +02)--MemoryExec: partitions=1, partition_sizes=[1] +03)--MemoryExec: partitions=1, partition_sizes=[1] # select NaNs query BBBB @@ -816,11 +816,11 @@ query TT EXPLAIN SELECT c1 BETWEEN 2 AND 3 FROM select_between_data ---- logical_plan -Projection: select_between_data.c1 >= Int64(2) AND select_between_data.c1 <= Int64(3) AS select_between_data.c1 BETWEEN Int64(2) AND Int64(3) ---TableScan: select_between_data projection=[c1] +01)Projection: select_between_data.c1 >= Int64(2) AND select_between_data.c1 <= Int64(3) AS select_between_data.c1 BETWEEN Int64(2) AND Int64(3) +02)--TableScan: select_between_data projection=[c1] physical_plan -ProjectionExec: expr=[c1@0 >= 2 AND c1@0 <= 3 as select_between_data.c1 BETWEEN Int64(2) AND Int64(3)] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[c1@0 >= 2 AND c1@0 <= 3 as select_between_data.c1 BETWEEN Int64(2) AND Int64(3)] +02)--MemoryExec: partitions=1, partition_sizes=[1] # TODO: query_get_indexed_field @@ -1081,12 +1081,12 @@ EXPLAIN SELECT a FROM annotated_data_finite2 LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: annotated_data_finite2.a ASC NULLS LAST, fetch=5 -----TableScan: annotated_data_finite2 projection=[a] +01)Limit: skip=0, fetch=5 +02)--Sort: annotated_data_finite2.a ASC NULLS LAST, fetch=5 +03)----TableScan: annotated_data_finite2 projection=[a] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true query I SELECT a FROM annotated_data_finite2 @@ -1208,11 +1208,11 @@ query TT EXPLAIN select a as a FROM table1 order by a ---- logical_plan -Sort: table1.a ASC NULLS LAST ---TableScan: table1 projection=[a] +01)Sort: table1.a ASC NULLS LAST +02)--TableScan: table1 projection=[a] physical_plan -SortExec: expr=[a@0 ASC NULLS LAST] ---MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[a@0 ASC NULLS LAST] +02)--MemoryExec: partitions=1, partition_sizes=[1] # ambiguous column references in on join query error DataFusion error: Schema error: Ambiguous reference to unqualified field a @@ -1231,14 +1231,14 @@ FROM annotated_data_finite2 ORDER BY a ASC; ---- logical_plan -Sort: annotated_data_finite2.a ASC NULLS LAST ---Projection: annotated_data_finite2.a, annotated_data_finite2.a + annotated_data_finite2.b -----TableScan: annotated_data_finite2 projection=[a, b] +01)Sort: annotated_data_finite2.a ASC NULLS LAST +02)--Projection: annotated_data_finite2.a, annotated_data_finite2.a + annotated_data_finite2.b +03)----TableScan: annotated_data_finite2 projection=[a, b] physical_plan -SortPreservingMergeExec: [a@0 ASC NULLS LAST] ---ProjectionExec: expr=[a@0 as a, a@0 + b@1 as annotated_data_finite2.a + annotated_data_finite2.b] -----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[a@0 as a, a@0 + b@1 as annotated_data_finite2.a + annotated_data_finite2.b] +03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # since query below doesn't computation # inside projection expr, increasing partitions @@ -1250,12 +1250,12 @@ FROM annotated_data_finite2 ORDER BY a ASC; ---- logical_plan -Sort: annotated_data_finite2.a ASC NULLS LAST ---Projection: annotated_data_finite2.a, annotated_data_finite2.b, Int64(2) -----TableScan: annotated_data_finite2 projection=[a, b] +01)Sort: annotated_data_finite2.a ASC NULLS LAST +02)--Projection: annotated_data_finite2.a, annotated_data_finite2.b, Int64(2) +03)----TableScan: annotated_data_finite2 projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, 2 as Int64(2)] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a, b@1 as b, 2 as Int64(2)] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # source is ordered by a,b,c # when filter result is constant for column a @@ -1268,15 +1268,15 @@ WHERE a=0 ORDER BY b, c; ---- logical_plan -Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST ---Filter: annotated_data_finite2.a = Int32(0) -----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] +01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] physical_plan -SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: a@1 = 0 -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: a@1 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1289,15 +1289,15 @@ WHERE a=0 and b=0 ORDER BY c; ---- logical_plan -Sort: annotated_data_finite2.c ASC NULLS LAST ---Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) -----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] +01)Sort: annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -SortPreservingMergeExec: [c@3 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: a@1 = 0 AND b@2 = 0 -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: a@1 = 0 AND b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1310,15 +1310,15 @@ WHERE a=0 and b=0 ORDER BY b, c; ---- logical_plan -Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST ---Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) -----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] +01)Sort: annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: a@1 = 0 AND b@2 = 0 -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: a@1 = 0 AND b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1331,15 +1331,15 @@ WHERE a=0 and b=0 ORDER BY a, b, c; ---- logical_plan -Sort: annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST ---Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) -----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] +01)Sort: annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -SortPreservingMergeExec: [a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: a@1 = 0 AND b@2 = 0 -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: a@1 = 0 AND b@2 = 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c # when filter result is when filter contains or @@ -1352,16 +1352,16 @@ WHERE a=0 or b=0 ORDER BY c; ---- logical_plan -Sort: annotated_data_finite2.c ASC NULLS LAST ---Filter: annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0) -----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0)] +01)Sort: annotated_data_finite2.c ASC NULLS LAST +02)--Filter: annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0) +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0) OR annotated_data_finite2.b = Int32(0)] physical_plan -SortPreservingMergeExec: [c@3 ASC NULLS LAST] ---SortExec: expr=[c@3 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 -------FilterExec: a@1 = 0 OR b@2 = 0 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] +02)--SortExec: expr=[c@3 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: a@1 = 0 OR b@2 = 0 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. # in the final physical plan. @@ -1373,18 +1373,18 @@ ORDER BY c1, c2) GROUP BY c2; ---- logical_plan -Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---Projection: aggregate_test_100.c2 -----Sort: aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST -------Projection: aggregate_test_100.c2, aggregate_test_100.c1 ---------TableScan: aggregate_test_100 projection=[c1, c2] +01)Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +02)--Projection: aggregate_test_100.c2 +03)----Sort: aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST +04)------Projection: aggregate_test_100.c2, aggregate_test_100.c1 +05)--------TableScan: aggregate_test_100 projection=[c1, c2] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[COUNT(*)] ---CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 -------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[COUNT(*)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true +01)AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[COUNT(*)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 +04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[COUNT(*)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true # FilterExec can track equality of non-column expressions. # plan below shouldn't have a SortExec because given column 'a' is ordered. @@ -1396,15 +1396,15 @@ WHERE CAST(ROUND(b) as INT) = a ORDER BY CAST(ROUND(b) as INT); ---- logical_plan -Sort: CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) ASC NULLS LAST ---Filter: CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a -----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a] +01)Sort: CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) ASC NULLS LAST +02)--Filter: CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a +03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[CAST(round(CAST(annotated_data_finite2.b AS Float64)) AS Int32) = annotated_data_finite2.a] physical_plan -SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] ---CoalesceBatchesExec: target_batch_size=8192 -----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true statement ok @@ -1426,13 +1426,13 @@ query TT EXPLAIN SELECT x/2, x/2+1 FROM t; ---- logical_plan -Projection: t.x / Int64(2)Int64(2)t.x AS t.x / Int64(2), t.x / Int64(2)Int64(2)t.x AS t.x / Int64(2) + Int64(1) ---Projection: t.x / Int64(2) AS t.x / Int64(2)Int64(2)t.x -----TableScan: t projection=[x] +01)Projection: t.x / Int64(2)Int64(2)t.x AS t.x / Int64(2), t.x / Int64(2)Int64(2)t.x AS t.x / Int64(2) + Int64(1) +02)--Projection: t.x / Int64(2) AS t.x / Int64(2)Int64(2)t.x +03)----TableScan: t projection=[x] physical_plan -ProjectionExec: expr=[t.x / Int64(2)Int64(2)t.x@0 as t.x / Int64(2), t.x / Int64(2)Int64(2)t.x@0 + 1 as t.x / Int64(2) + Int64(1)] ---ProjectionExec: expr=[x@0 / 2 as t.x / Int64(2)Int64(2)t.x] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t.x / Int64(2)Int64(2)t.x@0 as t.x / Int64(2), t.x / Int64(2)Int64(2)t.x@0 + 1 as t.x / Int64(2) + Int64(1)] +02)--ProjectionExec: expr=[x@0 / 2 as t.x / Int64(2)Int64(2)t.x] +03)----MemoryExec: partitions=1, partition_sizes=[1] query II SELECT x/2, x/2+1 FROM t; @@ -1444,13 +1444,13 @@ query TT EXPLAIN SELECT abs(x), abs(x) + abs(y) FROM t; ---- logical_plan -Projection: abs(t.x)t.x AS abs(t.x), abs(t.x)t.x AS abs(t.x) + abs(t.y) ---Projection: abs(t.x) AS abs(t.x)t.x, t.y -----TableScan: t projection=[x, y] +01)Projection: abs(t.x)t.x AS abs(t.x), abs(t.x)t.x AS abs(t.x) + abs(t.y) +02)--Projection: abs(t.x) AS abs(t.x)t.x, t.y +03)----TableScan: t projection=[x, y] physical_plan -ProjectionExec: expr=[abs(t.x)t.x@0 as abs(t.x), abs(t.x)t.x@0 + abs(y@1) as abs(t.x) + abs(t.y)] ---ProjectionExec: expr=[abs(x@0) as abs(t.x)t.x, y@1 as y] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[abs(t.x)t.x@0 as abs(t.x), abs(t.x)t.x@0 + abs(y@1) as abs(t.x) + abs(t.y)] +02)--ProjectionExec: expr=[abs(x@0) as abs(t.x)t.x, y@1 as y] +03)----MemoryExec: partitions=1, partition_sizes=[1] query II SELECT abs(x), abs(x) + abs(y) FROM t; @@ -1484,31 +1484,31 @@ query TT explain select coalesce(1, y/x), coalesce(2, y/x) from t; ---- logical_plan -Projection: coalesce(Int64(1), CAST(t.y / t.x AS Int64)), coalesce(Int64(2), CAST(t.y / t.x AS Int64)) ---TableScan: t projection=[x, y] +01)Projection: coalesce(Int64(1), CAST(t.y / t.x AS Int64)), coalesce(Int64(2), CAST(t.y / t.x AS Int64)) +02)--TableScan: t projection=[x, y] physical_plan -ProjectionExec: expr=[coalesce(1, CAST(y@1 / x@0 AS Int64)) as coalesce(Int64(1),t.y / t.x), coalesce(2, CAST(y@1 / x@0 AS Int64)) as coalesce(Int64(2),t.y / t.x)] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[coalesce(1, CAST(y@1 / x@0 AS Int64)) as coalesce(Int64(1),t.y / t.x), coalesce(2, CAST(y@1 / x@0 AS Int64)) as coalesce(Int64(2),t.y / t.x)] +02)--MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT y > 0 and 1 / y < 1, x > 0 and y > 0 and 1 / y < 1 / x from t; ---- logical_plan -Projection: t.y > Int32(0) AND Int64(1) / CAST(t.y AS Int64) < Int64(1) AS t.y > Int64(0) AND Int64(1) / t.y < Int64(1), t.x > Int32(0) AND t.y > Int32(0) AND Int64(1) / CAST(t.y AS Int64) < Int64(1) / CAST(t.x AS Int64) AS t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x ---TableScan: t projection=[x, y] +01)Projection: t.y > Int32(0) AND Int64(1) / CAST(t.y AS Int64) < Int64(1) AS t.y > Int64(0) AND Int64(1) / t.y < Int64(1), t.x > Int32(0) AND t.y > Int32(0) AND Int64(1) / CAST(t.y AS Int64) < Int64(1) / CAST(t.x AS Int64) AS t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x +02)--TableScan: t projection=[x, y] physical_plan -ProjectionExec: expr=[y@1 > 0 AND 1 / CAST(y@1 AS Int64) < 1 as t.y > Int64(0) AND Int64(1) / t.y < Int64(1), x@0 > 0 AND y@1 > 0 AND 1 / CAST(y@1 AS Int64) < 1 / CAST(x@0 AS Int64) as t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[y@1 > 0 AND 1 / CAST(y@1 AS Int64) < 1 as t.y > Int64(0) AND Int64(1) / t.y < Int64(1), x@0 > 0 AND y@1 > 0 AND 1 / CAST(y@1 AS Int64) < 1 / CAST(x@0 AS Int64) as t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x] +02)--MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT y = 0 or 1 / y < 1, x = 0 or y = 0 or 1 / y < 1 / x from t; ---- logical_plan -Projection: t.y = Int32(0) OR Int64(1) / CAST(t.y AS Int64) < Int64(1) AS t.y = Int64(0) OR Int64(1) / t.y < Int64(1), t.x = Int32(0) OR t.y = Int32(0) OR Int64(1) / CAST(t.y AS Int64) < Int64(1) / CAST(t.x AS Int64) AS t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x ---TableScan: t projection=[x, y] +01)Projection: t.y = Int32(0) OR Int64(1) / CAST(t.y AS Int64) < Int64(1) AS t.y = Int64(0) OR Int64(1) / t.y < Int64(1), t.x = Int32(0) OR t.y = Int32(0) OR Int64(1) / CAST(t.y AS Int64) < Int64(1) / CAST(t.x AS Int64) AS t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x +02)--TableScan: t projection=[x, y] physical_plan -ProjectionExec: expr=[y@1 = 0 OR 1 / CAST(y@1 AS Int64) < 1 as t.y = Int64(0) OR Int64(1) / t.y < Int64(1), x@0 = 0 OR y@1 = 0 OR 1 / CAST(y@1 AS Int64) < 1 / CAST(x@0 AS Int64) as t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[y@1 = 0 OR 1 / CAST(y@1 AS Int64) < 1 as t.y = Int64(0) OR Int64(1) / t.y < Int64(1), x@0 = 0 OR y@1 = 0 OR 1 / CAST(y@1 AS Int64) < 1 / CAST(x@0 AS Int64) as t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x] +02)--MemoryExec: partitions=1, partition_sizes=[1] # due to the reason describe in https://github.com/apache/arrow-datafusion/issues/8927, # the following queries will fail diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 426b9a3a5291..0361d4e5af7f 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -33,19 +33,19 @@ query TT EXPLAIN SELECT t1.a, t1.b, t2.a, t2.b FROM t1 JOIN t2 ON t1.a = t2.a AND t2.b * 50 <= t1.b ---- logical_plan -Inner Join: t1.a = t2.a Filter: CAST(t2.b AS Int64) * Int64(50) <= CAST(t1.b AS Int64) ---TableScan: t1 projection=[a, b] ---TableScan: t2 projection=[a, b] +01)Inner Join: t1.a = t2.a Filter: CAST(t2.b AS Int64) * Int64(50) <= CAST(t1.b AS Int64) +02)--TableScan: t1 projection=[a, b] +03)--TableScan: t2 projection=[a, b] physical_plan -SortMergeJoin: join_type=Inner, on=[(a@0, a@0)], filter=CAST(b@1 AS Int64) * 50 <= CAST(b@0 AS Int64) ---SortExec: expr=[a@0 ASC] -----CoalesceBatchesExec: target_batch_size=8192 -------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] ---SortExec: expr=[a@0 ASC] -----CoalesceBatchesExec: target_batch_size=8192 -------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)SortMergeJoin: join_type=Inner, on=[(a@0, a@0)], filter=CAST(b@1 AS Int64) * 50 <= CAST(b@0 AS Int64) +02)--SortExec: expr=[a@0 ASC] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--SortExec: expr=[a@0 ASC] +07)----CoalesceBatchesExec: target_batch_size=8192 +08)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +09)--------MemoryExec: partitions=1, partition_sizes=[1] # inner join with join filter query TITI rowsort diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index 8a6256add6ac..3e685cbb45a0 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -85,11 +85,11 @@ query TT explain select struct(a, b, c) from values; ---- logical_plan -Projection: struct(values.a, values.b, values.c) ---TableScan: values projection=[a, b, c] +01)Projection: struct(values.a, values.b, values.c) +02)--TableScan: values projection=[a, b, c] physical_plan -ProjectionExec: expr=[struct(a@0, b@1, c@2) as struct(values.a,values.b,values.c)] ---MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[struct(a@0, b@1, c@2) as struct(values.a,values.b,values.c)] +02)--MemoryExec: partitions=1, partition_sizes=[1] # error on 0 arguments query error DataFusion error: Error during planning: No function matches the given name and argument types 'named_struct\(\)'. You might need to add explicit type casts. diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 1ae89c9159f8..01f7dd904186 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -179,26 +179,26 @@ query TT explain SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ---Left Join: t1.t1_id = __scalar_sq_1.t2_id -----TableScan: t1 projection=[t1_id] -----SubqueryAlias: __scalar_sq_1 -------Projection: SUM(t2.t2_int), t2.t2_id ---------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] -----------TableScan: t2 projection=[t2_id, t2_int] +01)Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum +02)--Left Join: t1.t1_id = __scalar_sq_1.t2_id +03)----TableScan: t1 projection=[t1_id] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: SUM(t2.t2_int), t2.t2_id +06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] +07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int)@0, t1_id@2] -------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] ---------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +01)ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int)@0, t1_id@2] +04)------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +09)----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +10)------CoalesceBatchesExec: target_batch_size=2 +11)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +12)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -213,26 +213,26 @@ query TT explain SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2_sum ---Left Join: t1.t1_id = __scalar_sq_1.t2_id -----TableScan: t1 projection=[t1_id] -----SubqueryAlias: __scalar_sq_1 -------Projection: SUM(t2.t2_int * Float64(1)) + Float64(1) AS SUM(t2.t2_int * Float64(1)) + Int64(1), t2.t2_id ---------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Float64)) AS SUM(t2.t2_int * Float64(1))]] -----------TableScan: t2 projection=[t2_id, t2_int] +01)Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2_sum +02)--Left Join: t1.t1_id = __scalar_sq_1.t2_id +03)----TableScan: t1 projection=[t1_id] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: SUM(t2.t2_int * Float64(1)) + Float64(1) AS SUM(t2.t2_int * Float64(1)) + Int64(1), t2.t2_id +06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Float64)) AS SUM(t2.t2_int * Float64(1))]] +07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] -------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] ---------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +01)ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] +04)------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +09)----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +10)------CoalesceBatchesExec: target_batch_size=2 +11)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +12)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -247,28 +247,28 @@ query TT explain SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id group by t2_id, 'a') as t2_sum from t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ---Left Join: t1.t1_id = __scalar_sq_1.t2_id -----TableScan: t1 projection=[t1_id] -----SubqueryAlias: __scalar_sq_1 -------Projection: SUM(t2.t2_int), t2.t2_id ---------Aggregate: groupBy=[[t2.t2_id, Utf8("a")]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] -----------TableScan: t2 projection=[t2_id, t2_int] +01)Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum +02)--Left Join: t1.t1_id = __scalar_sq_1.t2_id +03)----TableScan: t1 projection=[t1_id] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: SUM(t2.t2_int), t2.t2_id +06)--------Aggregate: groupBy=[[t2.t2_id, Utf8("a")]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] +07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)], projection=[t1_id@0, SUM(t2.t2_int)@1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 -----------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] -------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) ---------------CoalesceBatchesExec: target_batch_size=2 -----------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 -------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) ---------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +01)ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)], projection=[t1_id@0, SUM(t2.t2_int)@1] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +06)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +07)------CoalesceBatchesExec: target_batch_size=2 +08)--------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 +09)----------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] +10)------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) +11)--------------CoalesceBatchesExec: target_batch_size=2 +12)----------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 +13)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) +14)--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id group by t2_id, 'a') as t2_sum from t1 @@ -283,29 +283,29 @@ query TT explain SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ---Left Join: t1.t1_id = __scalar_sq_1.t2_id -----TableScan: t1 projection=[t1_id] -----SubqueryAlias: __scalar_sq_1 -------Projection: SUM(t2.t2_int), t2.t2_id ---------Filter: SUM(t2.t2_int) < Int64(3) -----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] -------------TableScan: t2 projection=[t2_id, t2_int] +01)Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum +02)--Left Join: t1.t1_id = __scalar_sq_1.t2_id +03)----TableScan: t1 projection=[t1_id] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: SUM(t2.t2_int), t2.t2_id +06)--------Filter: SUM(t2.t2_int) < Int64(3) +07)----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] +08)------------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int)@0, t1_id@2] -------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] ---------CoalesceBatchesExec: target_batch_size=2 -----------FilterExec: SUM(t2.t2_int)@1 < 3 -------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------CoalesceBatchesExec: target_batch_size=2 -----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +01)ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int)@0, t1_id@2] +04)------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------FilterExec: SUM(t2.t2_int)@1 < 3 +07)------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +11)--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +12)------CoalesceBatchesExec: target_batch_size=2 +13)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +14)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 @@ -331,20 +331,20 @@ where c_acctbal < ( ) order by c_custkey; ---- logical_plan -Sort: customer.c_custkey ASC NULLS LAST ---Projection: customer.c_custkey -----Inner Join: customer.c_custkey = __scalar_sq_1.o_custkey Filter: CAST(customer.c_acctbal AS Decimal128(25, 2)) < __scalar_sq_1.SUM(orders.o_totalprice) -------TableScan: customer projection=[c_custkey, c_acctbal] -------SubqueryAlias: __scalar_sq_1 ---------Projection: SUM(orders.o_totalprice), orders.o_custkey -----------Aggregate: groupBy=[[orders.o_custkey]], aggr=[[SUM(orders.o_totalprice)]] -------------Projection: orders.o_custkey, orders.o_totalprice ---------------Inner Join: orders.o_orderkey = __scalar_sq_2.l_orderkey Filter: CAST(orders.o_totalprice AS Decimal128(25, 2)) < __scalar_sq_2.price -----------------TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice] -----------------SubqueryAlias: __scalar_sq_2 -------------------Projection: SUM(lineitem.l_extendedprice) AS price, lineitem.l_orderkey ---------------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_extendedprice)]] -----------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice] +01)Sort: customer.c_custkey ASC NULLS LAST +02)--Projection: customer.c_custkey +03)----Inner Join: customer.c_custkey = __scalar_sq_1.o_custkey Filter: CAST(customer.c_acctbal AS Decimal128(25, 2)) < __scalar_sq_1.SUM(orders.o_totalprice) +04)------TableScan: customer projection=[c_custkey, c_acctbal] +05)------SubqueryAlias: __scalar_sq_1 +06)--------Projection: SUM(orders.o_totalprice), orders.o_custkey +07)----------Aggregate: groupBy=[[orders.o_custkey]], aggr=[[SUM(orders.o_totalprice)]] +08)------------Projection: orders.o_custkey, orders.o_totalprice +09)--------------Inner Join: orders.o_orderkey = __scalar_sq_2.l_orderkey Filter: CAST(orders.o_totalprice AS Decimal128(25, 2)) < __scalar_sq_2.price +10)----------------TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice] +11)----------------SubqueryAlias: __scalar_sq_2 +12)------------------Projection: SUM(lineitem.l_extendedprice) AS price, lineitem.l_orderkey +13)--------------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_extendedprice)]] +14)----------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice] # correlated_where_in query TT @@ -354,12 +354,12 @@ where o_orderstatus in ( ); ---- logical_plan -Projection: orders.o_orderkey ---LeftSemi Join: orders.o_orderstatus = __correlated_sq_1.l_linestatus, orders.o_orderkey = __correlated_sq_1.l_orderkey -----TableScan: orders projection=[o_orderkey, o_orderstatus] -----SubqueryAlias: __correlated_sq_1 -------Projection: lineitem.l_linestatus, lineitem.l_orderkey ---------TableScan: lineitem projection=[l_orderkey, l_linestatus] +01)Projection: orders.o_orderkey +02)--LeftSemi Join: orders.o_orderstatus = __correlated_sq_1.l_linestatus, orders.o_orderkey = __correlated_sq_1.l_orderkey +03)----TableScan: orders projection=[o_orderkey, o_orderstatus] +04)----SubqueryAlias: __correlated_sq_1 +05)------Projection: lineitem.l_linestatus, lineitem.l_orderkey +06)--------TableScan: lineitem projection=[l_orderkey, l_linestatus] query I rowsort select o_orderkey from orders @@ -377,12 +377,12 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM t1 WHERE EXISTS(SELECT t1_int FROM t1 WHERE t1.t1_id > t1.t1_int) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Projection: t1.t1_int -------Filter: t1.t1_id > t1.t1_int ---------TableScan: t1 ---TableScan: t1 projection=[t1_id, t1_name, t1_int] +01)Filter: EXISTS () +02)--Subquery: +03)----Projection: t1.t1_int +04)------Filter: t1.t1_id > t1.t1_int +05)--------TableScan: t1 +06)--TableScan: t1 projection=[t1_id, t1_name, t1_int] #in_subquery_with_same_table @@ -392,26 +392,26 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM t1 WHERE t1_id IN(SELECT t1_int FROM t1 WHERE t1.t1_id > t1.t1_int) ---- logical_plan -LeftSemi Join: t1.t1_id = __correlated_sq_1.t1_int ---TableScan: t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: t1.t1_int -------Filter: t1.t1_int < t1.t1_id ---------TableScan: t1 projection=[t1_id, t1_int] +01)LeftSemi Join: t1.t1_id = __correlated_sq_1.t1_int +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: t1.t1_int +05)------Filter: t1.t1_int < t1.t1_id +06)--------TableScan: t1 projection=[t1_id, t1_int] #in_subquery_nested_exist_subquery query TT explain SELECT t1_id, t1_name, t1_int FROM t1 WHERE t1_id IN(SELECT t2_id FROM t2 WHERE EXISTS(select * from t1 WHERE t1.t1_int > t2.t2_int)) ---- logical_plan -LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id ---TableScan: t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: t2.t2_id -------LeftSemi Join: Filter: __correlated_sq_2.t1_int > t2.t2_int ---------TableScan: t2 projection=[t2_id, t2_int] ---------SubqueryAlias: __correlated_sq_2 -----------TableScan: t1 projection=[t1_int] +01)LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: t2.t2_id +05)------LeftSemi Join: Filter: __correlated_sq_2.t1_int > t2.t2_int +06)--------TableScan: t2 projection=[t2_id, t2_int] +07)--------SubqueryAlias: __correlated_sq_2 +08)----------TableScan: t1 projection=[t1_int] #invalid_scalar_subquery statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: Scalar subquery should only return one column, but found 2: t2.t2_id, t2.t2_name @@ -449,37 +449,37 @@ query TT explain SELECT t1_id, (SELECT t2_int FROM t2 WHERE t2.t2_int = t1.t1_int limit 1) as t2_int from t1 ---- logical_plan -Projection: t1.t1_id, () AS t2_int ---Subquery: -----Limit: skip=0, fetch=1 -------Projection: t2.t2_int ---------Filter: t2.t2_int = outer_ref(t1.t1_int) -----------TableScan: t2 ---TableScan: t1 projection=[t1_id, t1_int] +01)Projection: t1.t1_id, () AS t2_int +02)--Subquery: +03)----Limit: skip=0, fetch=1 +04)------Projection: t2.t2_int +05)--------Filter: t2.t2_int = outer_ref(t1.t1_int) +06)----------TableScan: t2 +07)--TableScan: t1 projection=[t1_id, t1_int] query TT explain SELECT t1_id from t1 where t1_int = (SELECT t2_int FROM t2 WHERE t2.t2_int = t1.t1_int limit 1) ---- logical_plan -Projection: t1.t1_id ---Filter: t1.t1_int = () -----Subquery: -------Limit: skip=0, fetch=1 ---------Projection: t2.t2_int -----------Filter: t2.t2_int = outer_ref(t1.t1_int) -------------TableScan: t2 -----TableScan: t1 projection=[t1_id, t1_int] +01)Projection: t1.t1_id +02)--Filter: t1.t1_int = () +03)----Subquery: +04)------Limit: skip=0, fetch=1 +05)--------Projection: t2.t2_int +06)----------Filter: t2.t2_int = outer_ref(t1.t1_int) +07)------------TableScan: t2 +08)----TableScan: t1 projection=[t1_id, t1_int] query TT explain SELECT t1_id, (SELECT a FROM (select 1 as a) WHERE a = t1.t1_int) as t2_int from t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.a AS t2_int ---Left Join: CAST(t1.t1_int AS Int64) = __scalar_sq_1.a -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: Int64(1) AS a ---------EmptyRelation +01)Projection: t1.t1_id, __scalar_sq_1.a AS t2_int +02)--Left Join: CAST(t1.t1_int AS Int64) = __scalar_sq_1.a +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: Int64(1) AS a +06)--------EmptyRelation query II rowsort SELECT t1_id, (SELECT a FROM (select 1 as a) WHERE a = t1.t1_int) as t2_int from t1 @@ -502,95 +502,95 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT sum(t1.t1_int + t2.t2_id) FROM t2 WHERE t1.t1_name = t2.t2_name) ---- logical_plan -Projection: t1.t1_id, t1.t1_name ---Filter: EXISTS () -----Subquery: -------Projection: SUM(outer_ref(t1.t1_int) + t2.t2_id) ---------Aggregate: groupBy=[[]], aggr=[[SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] -----------Filter: outer_ref(t1.t1_name) = t2.t2_name -------------TableScan: t2 -----TableScan: t1 projection=[t1_id, t1_name, t1_int] +01)Projection: t1.t1_id, t1.t1_name +02)--Filter: EXISTS () +03)----Subquery: +04)------Projection: SUM(outer_ref(t1.t1_int) + t2.t2_id) +05)--------Aggregate: groupBy=[[]], aggr=[[SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] +06)----------Filter: outer_ref(t1.t1_name) = t2.t2_name +07)------------TableScan: t2 +08)----TableScan: t1 projection=[t1_id, t1_name, t1_int] #support_agg_correlated_columns2 query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT count(*) FROM t2 WHERE t1.t1_name = t2.t2_name having sum(t1_int + t2_id) >0) ---- logical_plan -Projection: t1.t1_id, t1.t1_name ---Filter: EXISTS () -----Subquery: -------Projection: COUNT(*) ---------Filter: SUM(outer_ref(t1.t1_int) + t2.t2_id) > Int64(0) -----------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*), SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] -------------Filter: outer_ref(t1.t1_name) = t2.t2_name ---------------TableScan: t2 -----TableScan: t1 projection=[t1_id, t1_name, t1_int] +01)Projection: t1.t1_id, t1.t1_name +02)--Filter: EXISTS () +03)----Subquery: +04)------Projection: COUNT(*) +05)--------Filter: SUM(outer_ref(t1.t1_int) + t2.t2_id) > Int64(0) +06)----------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*), SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] +07)------------Filter: outer_ref(t1.t1_name) = t2.t2_name +08)--------------TableScan: t2 +09)----TableScan: t1 projection=[t1_id, t1_name, t1_int] #support_join_correlated_columns query TT explain SELECT t0_id, t0_name FROM t0 WHERE EXISTS (SELECT 1 FROM t1 INNER JOIN t2 ON(t1.t1_id = t2.t2_id and t1.t1_name = t0.t0_name)) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Projection: Int64(1) -------Inner Join: Filter: t1.t1_id = t2.t2_id AND t1.t1_name = outer_ref(t0.t0_name) ---------TableScan: t1 ---------TableScan: t2 ---TableScan: t0 projection=[t0_id, t0_name] +01)Filter: EXISTS () +02)--Subquery: +03)----Projection: Int64(1) +04)------Inner Join: Filter: t1.t1_id = t2.t2_id AND t1.t1_name = outer_ref(t0.t0_name) +05)--------TableScan: t1 +06)--------TableScan: t2 +07)--TableScan: t0 projection=[t0_id, t0_name] #subquery_contains_join_contains_correlated_columns query TT explain SELECT t0_id, t0_name FROM t0 WHERE EXISTS (SELECT 1 FROM t1 INNER JOIN (select * from t2 where t2.t2_name = t0.t0_name) as t2 ON(t1.t1_id = t2.t2_id )) ---- logical_plan -LeftSemi Join: t0.t0_name = __correlated_sq_1.t2_name ---TableScan: t0 projection=[t0_id, t0_name] ---SubqueryAlias: __correlated_sq_1 -----Projection: t2.t2_name -------Inner Join: t1.t1_id = t2.t2_id ---------TableScan: t1 projection=[t1_id] ---------SubqueryAlias: t2 -----------TableScan: t2 projection=[t2_id, t2_name] +01)LeftSemi Join: t0.t0_name = __correlated_sq_1.t2_name +02)--TableScan: t0 projection=[t0_id, t0_name] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: t2.t2_name +05)------Inner Join: t1.t1_id = t2.t2_id +06)--------TableScan: t1 projection=[t1_id] +07)--------SubqueryAlias: t2 +08)----------TableScan: t2 projection=[t2_id, t2_name] #subquery_contains_join_contains_sub_query_alias_correlated_columns query TT explain SELECT t0_id, t0_name FROM t0 WHERE EXISTS (select 1 from (SELECT * FROM t1 where t1.t1_id = t0.t0_id) as x INNER JOIN (select * from t2 where t2.t2_name = t0.t0_name) as y ON(x.t1_id = y.t2_id)) ---- logical_plan -LeftSemi Join: t0.t0_id = __correlated_sq_1.t1_id, t0.t0_name = __correlated_sq_1.t2_name ---TableScan: t0 projection=[t0_id, t0_name] ---SubqueryAlias: __correlated_sq_1 -----Projection: x.t1_id, y.t2_name -------Inner Join: x.t1_id = y.t2_id ---------SubqueryAlias: x -----------TableScan: t1 projection=[t1_id] ---------SubqueryAlias: y -----------TableScan: t2 projection=[t2_id, t2_name] +01)LeftSemi Join: t0.t0_id = __correlated_sq_1.t1_id, t0.t0_name = __correlated_sq_1.t2_name +02)--TableScan: t0 projection=[t0_id, t0_name] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: x.t1_id, y.t2_name +05)------Inner Join: x.t1_id = y.t2_id +06)--------SubqueryAlias: x +07)----------TableScan: t1 projection=[t1_id] +08)--------SubqueryAlias: y +09)----------TableScan: t2 projection=[t2_id, t2_name] #support_order_by_correlated_columns query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id >= t1_id order by t1_id) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Sort: outer_ref(t1.t1_id) ASC NULLS LAST -------Projection: t2.t2_id, t2.t2_name, t2.t2_int ---------Filter: t2.t2_id >= outer_ref(t1.t1_id) -----------TableScan: t2 ---TableScan: t1 projection=[t1_id, t1_name] +01)Filter: EXISTS () +02)--Subquery: +03)----Sort: outer_ref(t1.t1_id) ASC NULLS LAST +04)------Projection: t2.t2_id, t2.t2_name, t2.t2_int +05)--------Filter: t2.t2_id >= outer_ref(t1.t1_id) +06)----------TableScan: t2 +07)--TableScan: t1 projection=[t1_id, t1_name] #exists_subquery_with_select_null query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT NULL) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Projection: NULL -------EmptyRelation ---TableScan: t1 projection=[t1_id, t1_name] +01)Filter: EXISTS () +02)--Subquery: +03)----Projection: NULL +04)------EmptyRelation +05)--TableScan: t1 projection=[t1_id, t1_name] #exists_subquery_with_limit #de-correlated, limit is removed @@ -598,10 +598,10 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 1) ---- logical_plan -LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id ---TableScan: t1 projection=[t1_id, t1_name] ---SubqueryAlias: __correlated_sq_1 -----TableScan: t2 projection=[t2_id] +01)LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id +02)--TableScan: t1 projection=[t1_id, t1_name] +03)--SubqueryAlias: __correlated_sq_1 +04)----TableScan: t2 projection=[t2_id] query IT rowsort SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 1) @@ -616,9 +616,9 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 0) ---- logical_plan -LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id ---TableScan: t1 projection=[t1_id, t1_name] ---EmptyRelation +01)LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id +02)--TableScan: t1 projection=[t1_id, t1_name] +03)--EmptyRelation query IT rowsort SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 0) @@ -631,9 +631,9 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE NOT EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 0) ---- logical_plan -LeftAnti Join: t1.t1_id = __correlated_sq_1.t2_id ---TableScan: t1 projection=[t1_id, t1_name] ---EmptyRelation +01)LeftAnti Join: t1.t1_id = __correlated_sq_1.t2_id +02)--TableScan: t1 projection=[t1_id, t1_name] +03)--EmptyRelation query IT rowsort SELECT t1_id, t1_name FROM t1 WHERE NOT EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id limit 0) @@ -649,13 +649,13 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE t1_id in (SELECT t2_id FROM t2 where t1_name = t2_name limit 10) ---- logical_plan -Filter: t1.t1_id IN () ---Subquery: -----Limit: skip=0, fetch=10 -------Projection: t2.t2_id ---------Filter: outer_ref(t1.t1_name) = t2.t2_name -----------TableScan: t2 ---TableScan: t1 projection=[t1_id, t1_name] +01)Filter: t1.t1_id IN () +02)--Subquery: +03)----Limit: skip=0, fetch=10 +04)------Projection: t2.t2_id +05)--------Filter: outer_ref(t1.t1_name) = t2.t2_name +06)----------TableScan: t2 +07)--TableScan: t1 projection=[t1_id, t1_name] #in_non_correlated_subquery_with_limit #de-correlated, limit is kept @@ -663,11 +663,11 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE t1_id in (SELECT t2_id FROM t2 limit 10) ---- logical_plan -LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id ---TableScan: t1 projection=[t1_id, t1_name] ---SubqueryAlias: __correlated_sq_1 -----Limit: skip=0, fetch=10 -------TableScan: t2 projection=[t2_id], fetch=10 +01)LeftSemi Join: t1.t1_id = __correlated_sq_1.t2_id +02)--TableScan: t1 projection=[t1_id, t1_name] +03)--SubqueryAlias: __correlated_sq_1 +04)----Limit: skip=0, fetch=10 +05)------TableScan: t2 projection=[t2_id], fetch=10 #uncorrelated_scalar_subquery_with_limit0 @@ -675,10 +675,10 @@ query TT explain SELECT t1_id, (SELECT t2_id FROM t2 limit 0) FROM t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.t2_id AS t2_id ---Left Join: -----TableScan: t1 projection=[t1_id] -----EmptyRelation +01)Projection: t1.t1_id, __scalar_sq_1.t2_id AS t2_id +02)--Left Join: +03)----TableScan: t1 projection=[t1_id] +04)----EmptyRelation query II rowsort SELECT t1_id, (SELECT t2_id FROM t2 limit 0) FROM t1 @@ -693,40 +693,40 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT * FROM t2 WHERE t2_id = t1_id UNION ALL SELECT * FROM t2 WHERE upper(t2_name) = upper(t1.t1_name)) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Union -------Projection: t2.t2_id, t2.t2_name, t2.t2_int ---------Filter: t2.t2_id = outer_ref(t1.t1_id) -----------TableScan: t2 -------Projection: t2.t2_id, t2.t2_name, t2.t2_int ---------Filter: upper(t2.t2_name) = upper(outer_ref(t1.t1_name)) -----------TableScan: t2 ---TableScan: t1 projection=[t1_id, t1_name] +01)Filter: EXISTS () +02)--Subquery: +03)----Union +04)------Projection: t2.t2_id, t2.t2_name, t2.t2_int +05)--------Filter: t2.t2_id = outer_ref(t1.t1_id) +06)----------TableScan: t2 +07)------Projection: t2.t2_id, t2.t2_name, t2.t2_int +08)--------Filter: upper(t2.t2_name) = upper(outer_ref(t1.t1_name)) +09)----------TableScan: t2 +10)--TableScan: t1 projection=[t1_id, t1_name] #simple_uncorrelated_scalar_subquery query TT explain select (select count(*) from t1) as b ---- logical_plan -Projection: __scalar_sq_1.COUNT(*) AS b ---SubqueryAlias: __scalar_sq_1 -----Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -------TableScan: t1 projection=[] +01)Projection: __scalar_sq_1.COUNT(*) AS b +02)--SubqueryAlias: __scalar_sq_1 +03)----Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------TableScan: t1 projection=[] #simple_uncorrelated_scalar_subquery2 query TT explain select (select count(*) from t1) as b, (select count(1) from t2) ---- logical_plan -Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT(Int64(1)) ---Left Join: -----SubqueryAlias: __scalar_sq_1 -------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------TableScan: t1 projection=[] -----SubqueryAlias: __scalar_sq_2 -------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] ---------TableScan: t2 projection=[] +01)Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT(Int64(1)) +02)--Left Join: +03)----SubqueryAlias: __scalar_sq_1 +04)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)--------TableScan: t1 projection=[] +06)----SubqueryAlias: __scalar_sq_2 +07)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] +08)--------TableScan: t2 projection=[] query II select (select count(*) from t1) as b, (select count(1) from t2) @@ -738,13 +738,13 @@ query TT explain SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) from t1 ---- logical_plan -Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END AS COUNT(*) ---Left Join: t1.t1_int = __scalar_sq_1.t2_int -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: COUNT(*), t2.t2_int, __always_true ---------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END AS COUNT(*) +02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: COUNT(*), t2.t2_int, __always_true +06)--------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------TableScan: t2 projection=[t2_int] query II rowsort SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) from t1 @@ -760,13 +760,13 @@ query TT explain SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) as cnt from t1 ---- logical_plan -Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END AS cnt ---Left Join: t1.t1_int = __scalar_sq_1.t2_int -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: COUNT(*), t2.t2_int, __always_true ---------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END AS cnt +02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: COUNT(*), t2.t2_int, __always_true +06)--------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------TableScan: t2 projection=[t2_int] query II rowsort SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) as cnt from t1 @@ -781,13 +781,13 @@ query TT explain SELECT t1_id, (SELECT count(*) as _cnt FROM t2 WHERE t2.t2_int = t1.t1_int) as cnt from t1 ---- logical_plan -Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) AS _cnt ELSE __scalar_sq_1._cnt END AS cnt ---Left Join: t1.t1_int = __scalar_sq_1.t2_int -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: COUNT(*) AS _cnt, t2.t2_int, __always_true ---------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) AS _cnt ELSE __scalar_sq_1._cnt END AS cnt +02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: COUNT(*) AS _cnt, t2.t2_int, __always_true +06)--------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------TableScan: t2 projection=[t2_int] query II rowsort SELECT t1_id, (SELECT count(*) as _cnt FROM t2 WHERE t2.t2_int = t1.t1_int) as cnt from t1 @@ -802,13 +802,13 @@ query TT explain SELECT t1_id, (SELECT count(*) + 2 as _cnt FROM t2 WHERE t2.t2_int = t1.t1_int) from t1 ---- logical_plan -Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) AS _cnt ELSE __scalar_sq_1._cnt END AS _cnt ---Left Join: t1.t1_int = __scalar_sq_1.t2_int -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: COUNT(*) + Int64(2) AS _cnt, t2.t2_int, __always_true ---------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) AS _cnt ELSE __scalar_sq_1._cnt END AS _cnt +02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: COUNT(*) + Int64(2) AS _cnt, t2.t2_int, __always_true +06)--------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------TableScan: t2 projection=[t2_int] query II rowsort SELECT t1_id, (SELECT count(*) + 2 as _cnt FROM t2 WHERE t2.t2_int = t1.t1_int) from t1 @@ -823,15 +823,15 @@ query TT explain select t1.t1_int from t1 where (select count(*) from t2 where t1.t1_id = t2.t2_id) < t1.t1_int ---- logical_plan -Projection: t1.t1_int ---Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END < CAST(t1.t1_int AS Int64) -----Projection: t1.t1_int, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true -------Left Join: t1.t1_id = __scalar_sq_1.t2_id ---------TableScan: t1 projection=[t1_id, t1_int] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: COUNT(*), t2.t2_id, __always_true -------------Aggregate: groupBy=[[t2.t2_id, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------------TableScan: t2 projection=[t2_id] +01)Projection: t1.t1_int +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END < CAST(t1.t1_int AS Int64) +03)----Projection: t1.t1_int, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true +04)------Left Join: t1.t1_id = __scalar_sq_1.t2_id +05)--------TableScan: t1 projection=[t1_id, t1_int] +06)--------SubqueryAlias: __scalar_sq_1 +07)----------Projection: COUNT(*), t2.t2_id, __always_true +08)------------Aggregate: groupBy=[[t2.t2_id, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +09)--------------TableScan: t2 projection=[t2_id] query I rowsort select t1.t1_int from t1 where (select count(*) from t2 where t1.t1_id = t2.t2_id) < t1.t1_int @@ -846,15 +846,15 @@ query TT explain SELECT t1_id, (SELECT count(*) + 2 as cnt_plus_2 FROM t2 WHERE t2.t2_int = t1.t1_int having count(*) >1) from t1 ---- logical_plan -Projection: t1.t1_id, __scalar_sq_1.cnt_plus_2 AS cnt_plus_2 ---Left Join: t1.t1_int = __scalar_sq_1.t2_int -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: COUNT(*) + Int64(2) AS cnt_plus_2, t2.t2_int ---------Filter: COUNT(*) > Int64(1) -----------Projection: t2.t2_int, COUNT(*) -------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_id, __scalar_sq_1.cnt_plus_2 AS cnt_plus_2 +02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: COUNT(*) + Int64(2) AS cnt_plus_2, t2.t2_int +06)--------Filter: COUNT(*) > Int64(1) +07)----------Projection: t2.t2_int, COUNT(*) +08)------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +09)--------------TableScan: t2 projection=[t2_int] query II rowsort SELECT t1_id, (SELECT count(*) + 2 as cnt_plus_2 FROM t2 WHERE t2.t2_int = t1.t1_int having count(*) >1) from t1 @@ -870,13 +870,13 @@ query TT explain SELECT t1_id, (SELECT count(*) + 2 as cnt_plus_2 FROM t2 WHERE t2.t2_int = t1.t1_int having count(*) = 0) from t1 ---- logical_plan -Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) AS cnt_plus_2 WHEN __scalar_sq_1.COUNT(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_2 END AS cnt_plus_2 ---Left Join: t1.t1_int = __scalar_sq_1.t2_int -----TableScan: t1 projection=[t1_id, t1_int] -----SubqueryAlias: __scalar_sq_1 -------Projection: COUNT(*) + Int64(2) AS cnt_plus_2, t2.t2_int, COUNT(*), __always_true ---------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_id, CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) AS cnt_plus_2 WHEN __scalar_sq_1.COUNT(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_2 END AS cnt_plus_2 +02)--Left Join: t1.t1_int = __scalar_sq_1.t2_int +03)----TableScan: t1 projection=[t1_id, t1_int] +04)----SubqueryAlias: __scalar_sq_1 +05)------Projection: COUNT(*) + Int64(2) AS cnt_plus_2, t2.t2_int, COUNT(*), __always_true +06)--------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +07)----------TableScan: t2 projection=[t2_int] query II rowsort SELECT t1_id, (SELECT count(*) + 2 as cnt_plus_2 FROM t2 WHERE t2.t2_int = t1.t1_int having count(*) = 0) from t1 @@ -891,16 +891,16 @@ query TT explain select t1.t1_int from t1 group by t1.t1_int having (select count(*) from t2 where t1.t1_int = t2.t2_int) = 0 ---- logical_plan -Projection: t1.t1_int ---Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END = Int64(0) -----Projection: t1.t1_int, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true -------Left Join: t1.t1_int = __scalar_sq_1.t2_int ---------Aggregate: groupBy=[[t1.t1_int]], aggr=[[]] -----------TableScan: t1 projection=[t1_int] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: COUNT(*), t2.t2_int, __always_true -------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_int +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.COUNT(*) END = Int64(0) +03)----Projection: t1.t1_int, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true +04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int +05)--------Aggregate: groupBy=[[t1.t1_int]], aggr=[[]] +06)----------TableScan: t1 projection=[t1_int] +07)--------SubqueryAlias: __scalar_sq_1 +08)----------Projection: COUNT(*), t2.t2_int, __always_true +09)------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +10)--------------TableScan: t2 projection=[t2_int] query I rowsort select t1.t1_int from t1 group by t1.t1_int having (select count(*) from t2 where t1.t1_int = t2.t2_int) = 0 @@ -913,15 +913,15 @@ query TT explain select t1.t1_int from t1 where (select cnt from (select count(*) as cnt, sum(t2_int) from t2 where t1.t1_int = t2.t2_int)) = 0 ---- logical_plan -Projection: t1.t1_int ---Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.cnt END = Int64(0) -----Projection: t1.t1_int, __scalar_sq_1.cnt, __scalar_sq_1.__always_true -------Left Join: t1.t1_int = __scalar_sq_1.t2_int ---------TableScan: t1 projection=[t1_int] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: COUNT(*) AS cnt, t2.t2_int, __always_true -------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_int +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.cnt END = Int64(0) +03)----Projection: t1.t1_int, __scalar_sq_1.cnt, __scalar_sq_1.__always_true +04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int +05)--------TableScan: t1 projection=[t1_int] +06)--------SubqueryAlias: __scalar_sq_1 +07)----------Projection: COUNT(*) AS cnt, t2.t2_int, __always_true +08)------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +09)--------------TableScan: t2 projection=[t2_int] query I rowsort @@ -943,15 +943,15 @@ select t1.t1_int from t1 where ( ) = 2 ---- logical_plan -Projection: t1.t1_int ---Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) WHEN __scalar_sq_1.COUNT(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_two END = Int64(2) -----Projection: t1.t1_int, __scalar_sq_1.cnt_plus_two, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true -------Left Join: t1.t1_int = __scalar_sq_1.t2_int ---------TableScan: t1 projection=[t1_int] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: COUNT(*) + Int64(1) + Int64(1) AS cnt_plus_two, t2.t2_int, COUNT(*), __always_true -------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_int +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(2) WHEN __scalar_sq_1.COUNT(*) != Int64(0) THEN NULL ELSE __scalar_sq_1.cnt_plus_two END = Int64(2) +03)----Projection: t1.t1_int, __scalar_sq_1.cnt_plus_two, __scalar_sq_1.COUNT(*), __scalar_sq_1.__always_true +04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int +05)--------TableScan: t1 projection=[t1_int] +06)--------SubqueryAlias: __scalar_sq_1 +07)----------Projection: COUNT(*) + Int64(1) + Int64(1) AS cnt_plus_two, t2.t2_int, COUNT(*), __always_true +08)------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +09)--------------TableScan: t2 projection=[t2_int] query I rowsort select t1.t1_int from t1 where ( @@ -972,15 +972,15 @@ select t1.t1_int from t1 where (select case when count(*) = 1 then null else count(*) end as cnt from t2 where t2.t2_int = t1.t1_int) = 0 ---- logical_plan -Projection: t1.t1_int ---Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.cnt END = Int64(0) -----Projection: t1.t1_int, __scalar_sq_1.cnt, __scalar_sq_1.__always_true -------Left Join: t1.t1_int = __scalar_sq_1.t2_int ---------TableScan: t1 projection=[t1_int] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: CASE WHEN COUNT(*) = Int64(1) THEN Int64(NULL) ELSE COUNT(*) END AS cnt, t2.t2_int, __always_true -------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------------TableScan: t2 projection=[t2_int] +01)Projection: t1.t1_int +02)--Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.cnt END = Int64(0) +03)----Projection: t1.t1_int, __scalar_sq_1.cnt, __scalar_sq_1.__always_true +04)------Left Join: t1.t1_int = __scalar_sq_1.t2_int +05)--------TableScan: t1 projection=[t1_int] +06)--------SubqueryAlias: __scalar_sq_1 +07)----------Projection: CASE WHEN COUNT(*) = Int64(1) THEN Int64(NULL) ELSE COUNT(*) END AS cnt, t2.t2_int, __always_true +08)------------Aggregate: groupBy=[[t2.t2_int, Boolean(true) AS __always_true]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +09)--------------TableScan: t2 projection=[t2_int] query I rowsort @@ -1046,9 +1046,9 @@ query TT explain select a/2, a/2 + 1 from t ---- logical_plan -Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) ---Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a -----TableScan: t projection=[a] +01)Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) +02)--Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a +03)----TableScan: t projection=[a] statement ok set datafusion.optimizer.max_passes = 3; @@ -1057,9 +1057,9 @@ query TT explain select a/2, a/2 + 1 from t ---- logical_plan -Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) ---Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a -----TableScan: t projection=[a] +01)Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) +02)--Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a +03)----TableScan: t projection=[a] ### ## Ensure that operators are rewritten in subqueries diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index f0e04b522a78..491b9b810687 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2794,4 +2794,4 @@ SELECT '2000-12-01 04:04:12' AT TIME ZONE 'America/New York'; # abbreviated timezone is not supported statement error -SELECT '2023-03-12 02:00:00' AT TIME ZONE 'EDT'; \ No newline at end of file +SELECT '2023-03-12 02:00:00' AT TIME ZONE 'EDT'; diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 5eba20fdc655..1035c49d1fc0 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -76,13 +76,13 @@ query TT explain select * from aggregate_test_100 ORDER BY c13 desc limit 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c13 DESC NULLS FIRST, fetch=5 -----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c13 DESC NULLS FIRST, fetch=5 +03)----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c13@12 DESC] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c13@12 DESC] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 52ad63e99f70..175040420160 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -40,24 +40,24 @@ explain select l_linestatus; ---- logical_plan -Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST ---Projection: lineitem.l_returnflag, lineitem.l_linestatus, SUM(lineitem.l_quantity) AS sum_qty, SUM(lineitem.l_extendedprice) AS sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(*) AS count_order -----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(Int64(1)) AS COUNT(*)]] -------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus ---------Filter: lineitem.l_shipdate <= Date32("10471") -----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("10471")] +01)Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST +02)--Projection: lineitem.l_returnflag, lineitem.l_linestatus, SUM(lineitem.l_quantity) AS sum_qty, SUM(lineitem.l_extendedprice) AS sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(*) AS count_order +03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(Int64(1)) AS COUNT(*)]] +04)------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus +05)--------Filter: lineitem.l_shipdate <= Date32("10471") +06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("10471")] physical_plan -SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] ---SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] -----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(*)@9 as count_order] -------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] ---------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------FilterExec: l_shipdate@6 <= 10471 ---------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false +01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] +02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] +03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(*)@9 as count_order] +04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] +08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------FilterExec: l_shipdate@6 <= 10471 +11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false query TTRRRRRRRI select diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index 47c9ea2d005c..2a3168b5c1bf 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -51,63 +51,63 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: revenue DESC NULLS FIRST, fetch=10 -----Projection: customer.c_custkey, customer.c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment -------Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] ---------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name -----------Inner Join: customer.c_nationkey = nation.n_nationkey -------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount ---------------Inner Join: orders.o_orderkey = lineitem.l_orderkey -----------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey -------------------Inner Join: customer.c_custkey = orders.o_custkey ---------------------TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] ---------------------Projection: orders.o_orderkey, orders.o_custkey -----------------------Filter: orders.o_orderdate >= Date32("8674") AND orders.o_orderdate < Date32("8766") -------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8674"), orders.o_orderdate < Date32("8766")] -----------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount -------------------Filter: lineitem.l_returnflag = Utf8("R") ---------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8("R")] -------------TableScan: nation projection=[n_nationkey, n_name] +01)Limit: skip=0, fetch=10 +02)--Sort: revenue DESC NULLS FIRST, fetch=10 +03)----Projection: customer.c_custkey, customer.c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment +04)------Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +05)--------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name +06)----------Inner Join: customer.c_nationkey = nation.n_nationkey +07)------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount +08)--------------Inner Join: orders.o_orderkey = lineitem.l_orderkey +09)----------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey +10)------------------Inner Join: customer.c_custkey = orders.o_custkey +11)--------------------TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] +12)--------------------Projection: orders.o_orderkey, orders.o_custkey +13)----------------------Filter: orders.o_orderdate >= Date32("8674") AND orders.o_orderdate < Date32("8766") +14)------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8674"), orders.o_orderdate < Date32("8766")] +15)----------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +16)------------------Filter: lineitem.l_returnflag = Utf8("R") +17)--------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8("R")] +18)------------TableScan: nation projection=[n_nationkey, n_name] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [revenue@2 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[revenue@2 DESC] -------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] ---------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 -----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: l_returnflag@3 = R ---------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [revenue@2 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[revenue@2 DESC] +04)------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] +05)--------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +21)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false +23)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +25)----------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +26)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 +28)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +29)----------------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +31)--------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +32)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------------FilterExec: l_returnflag@3 = R +34)--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false +35)--------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +37)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 9e3d0e6d37ae..55b38333ca38 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -47,86 +47,86 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: value DESC NULLS FIRST, fetch=10 -----Projection: partsupp.ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS value -------Inner Join: Filter: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > __scalar_sq_1.SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001) ---------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] -----------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost -------------Inner Join: supplier.s_nationkey = nation.n_nationkey ---------------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey -----------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey -------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] -------------------TableScan: supplier projection=[s_suppkey, s_nationkey] ---------------Projection: nation.n_nationkey -----------------Filter: nation.n_name = Utf8("GERMANY") -------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) -------------Aggregate: groupBy=[[]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] ---------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost -----------------Inner Join: supplier.s_nationkey = nation.n_nationkey -------------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey ---------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey -----------------------TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] -----------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -------------------Projection: nation.n_nationkey ---------------------Filter: nation.n_name = Utf8("GERMANY") -----------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] +01)Limit: skip=0, fetch=10 +02)--Sort: value DESC NULLS FIRST, fetch=10 +03)----Projection: partsupp.ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS value +04)------Inner Join: Filter: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > __scalar_sq_1.SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001) +05)--------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] +06)----------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost +07)------------Inner Join: supplier.s_nationkey = nation.n_nationkey +08)--------------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey +09)----------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +10)------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] +11)------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +12)--------------Projection: nation.n_nationkey +13)----------------Filter: nation.n_name = Utf8("GERMANY") +14)------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] +15)--------SubqueryAlias: __scalar_sq_1 +16)----------Projection: CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) +17)------------Aggregate: groupBy=[[]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] +18)--------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost +19)----------------Inner Join: supplier.s_nationkey = nation.n_nationkey +20)------------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey +21)--------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +22)----------------------TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] +23)----------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +24)------------------Projection: nation.n_nationkey +25)--------------------Filter: nation.n_name = Utf8("GERMANY") +26)----------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [value@1 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[value@1 DESC] -------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] ---------NestedLoopJoinExec: join_type=Inner, filter=CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 -----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: n_name@1 = GERMANY ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: n_name@1 = GERMANY ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [value@1 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[value@1 DESC] +04)------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] +05)--------NestedLoopJoinExec: join_type=Inner, filter=CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 +06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +09)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +18)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false +19)------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +21)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +22)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +23)----------------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +25)--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +26)----------------------------CoalesceBatchesExec: target_batch_size=8192 +27)------------------------------FilterExec: n_name@1 = GERMANY +28)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +30)----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +31)------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)--------------CoalescePartitionsExec +33)----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +38)--------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +40)------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +42)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false +43)------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +45)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +49)--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +50)----------------------------CoalesceBatchesExec: target_batch_size=8192 +51)------------------------------FilterExec: n_name@1 = GERMANY +52)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +53)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/q12.slt.part index 7930d36aac65..95cde1b0f105 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -49,34 +49,34 @@ order by l_shipmode; ---- logical_plan -Sort: lineitem.l_shipmode ASC NULLS LAST ---Projection: lineitem.l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS low_line_count -----Aggregate: groupBy=[[lineitem.l_shipmode]], aggr=[[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]] -------Projection: lineitem.l_shipmode, orders.o_orderpriority ---------Inner Join: lineitem.l_orderkey = orders.o_orderkey -----------Projection: lineitem.l_orderkey, lineitem.l_shipmode -------------Filter: (lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("8766") AND lineitem.l_receiptdate < Date32("9131") ---------------TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("8766"), lineitem.l_receiptdate < Date32("9131")] -----------TableScan: orders projection=[o_orderkey, o_orderpriority] +01)Sort: lineitem.l_shipmode ASC NULLS LAST +02)--Projection: lineitem.l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS low_line_count +03)----Aggregate: groupBy=[[lineitem.l_shipmode]], aggr=[[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]] +04)------Projection: lineitem.l_shipmode, orders.o_orderpriority +05)--------Inner Join: lineitem.l_orderkey = orders.o_orderkey +06)----------Projection: lineitem.l_orderkey, lineitem.l_shipmode +07)------------Filter: (lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("8766") AND lineitem.l_receiptdate < Date32("9131") +08)--------------TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("8766"), lineitem.l_receiptdate < Date32("9131")] +09)----------TableScan: orders projection=[o_orderkey, o_orderpriority] physical_plan -SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] ---SortExec: expr=[l_shipmode@0 ASC NULLS LAST] -----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] -------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 8766 AND l_receiptdate@3 < 9131 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false +01)SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] +02)--SortExec: expr=[l_shipmode@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] +04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +12)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 8766 AND l_receiptdate@3 < 9131 +15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +18)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index 592a50790419..cf4983fb0431 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -40,42 +40,42 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST, fetch=10 -----Projection: c_orders.c_count, COUNT(*) AS custdist -------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------SubqueryAlias: c_orders -----------Projection: COUNT(orders.o_orderkey) AS c_count -------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] ---------------Projection: customer.c_custkey, orders.o_orderkey -----------------Left Join: customer.c_custkey = orders.o_custkey -------------------TableScan: customer projection=[c_custkey] -------------------Projection: orders.o_orderkey, orders.o_custkey ---------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") -----------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] +01)Limit: skip=0, fetch=10 +02)--Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST, fetch=10 +03)----Projection: c_orders.c_count, COUNT(*) AS custdist +04)------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)--------SubqueryAlias: c_orders +06)----------Projection: COUNT(orders.o_orderkey) AS c_count +07)------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] +08)--------------Projection: customer.c_custkey, orders.o_orderkey +09)----------------Left Join: customer.c_custkey = orders.o_custkey +10)------------------TableScan: customer projection=[c_custkey] +11)------------------Projection: orders.o_orderkey, orders.o_custkey +12)--------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") +13)----------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC] -------ProjectionExec: expr=[c_count@0 as c_count, COUNT(*)@1 as custdist] ---------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(*)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] -----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] -------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC] +04)------ProjectionExec: expr=[c_count@0 as c_count, COUNT(*)@1 as custdist] +05)--------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(*)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] +09)----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] +10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +17)------------------------CoalesceBatchesExec: target_batch_size=8192 +18)--------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +19)----------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +20)------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)--------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% +22)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/q14.slt.part index 8d6864674f64..3d598fdfb63a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -32,31 +32,31 @@ where and l_shipdate < date '1995-10-01'; ---- logical_plan -Projection: Float64(100) * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END) AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS Float64) AS promo_revenue ---Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_type -------Inner Join: lineitem.l_partkey = part.p_partkey ---------Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount -----------Filter: lineitem.l_shipdate >= Date32("9374") AND lineitem.l_shipdate < Date32("9404") -------------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9374"), lineitem.l_shipdate < Date32("9404")] ---------TableScan: part projection=[p_partkey, p_type] +01)Projection: Float64(100) * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END) AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS Float64) AS promo_revenue +02)--Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +03)----Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_type +04)------Inner Join: lineitem.l_partkey = part.p_partkey +05)--------Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount +06)----------Filter: lineitem.l_shipdate >= Date32("9374") AND lineitem.l_shipdate < Date32("9404") +07)------------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9374"), lineitem.l_shipdate < Date32("9404")] +08)--------TableScan: part projection=[p_partkey, p_type] physical_plan -ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 -----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +01)ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 +12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false +13)------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 2b5f74186385..c01cc52ce4bc 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -50,62 +50,62 @@ order by s_suppkey; ---- logical_plan -Sort: supplier.s_suppkey ASC NULLS LAST ---Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue -----Inner Join: revenue0.total_revenue = __scalar_sq_1.MAX(revenue0.total_revenue) -------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue ---------Inner Join: supplier.s_suppkey = revenue0.supplier_no -----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] -----------SubqueryAlias: revenue0 -------------Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue ---------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") ---------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] -------SubqueryAlias: __scalar_sq_1 ---------Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] -----------SubqueryAlias: revenue0 -------------Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue ---------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") ---------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] +01)Sort: supplier.s_suppkey ASC NULLS LAST +02)--Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue +03)----Inner Join: revenue0.total_revenue = __scalar_sq_1.MAX(revenue0.total_revenue) +04)------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue +05)--------Inner Join: supplier.s_suppkey = revenue0.supplier_no +06)----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] +07)----------SubqueryAlias: revenue0 +08)------------Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue +09)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +10)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +11)------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") +12)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] +13)------SubqueryAlias: __scalar_sq_1 +14)--------Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] +15)----------SubqueryAlias: revenue0 +16)------------Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue +17)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +18)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +19)------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") +20)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] physical_plan -SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ---SortExec: expr=[s_suppkey@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false -----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ---------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 -------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] -------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] ---------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 ---------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] +02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +13)----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +14)------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +15)--------------------CoalesceBatchesExec: target_batch_size=8192 +16)----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +17)------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +18)--------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +22)--------CoalesceBatchesExec: target_batch_size=8192 +23)----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +24)------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +25)--------------CoalescePartitionsExec +26)----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +27)------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +29)----------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +31)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +32)----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +33)------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +35)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 77292a12221a..16f808765228 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -50,56 +50,56 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 -----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt -------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] ---------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] -----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey -------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size ---------------Inner Join: partsupp.ps_partkey = part.p_partkey -----------------TableScan: partsupp projection=[ps_partkey, ps_suppkey] -----------------Filter: part.p_brand != Utf8("Brand#45") AND part.p_type NOT LIKE Utf8("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) -------------------TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#45"), part.p_type NOT LIKE Utf8("MEDIUM POLISHED%"), part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)])] -------------SubqueryAlias: __correlated_sq_1 ---------------Projection: supplier.s_suppkey -----------------Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") -------------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] +01)Limit: skip=0, fetch=10 +02)--Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 +03)----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt +04)------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] +05)--------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] +06)----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey +07)------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size +08)--------------Inner Join: partsupp.ps_partkey = part.p_partkey +09)----------------TableScan: partsupp projection=[ps_partkey, ps_suppkey] +10)----------------Filter: part.p_brand != Utf8("Brand#45") AND part.p_type NOT LIKE Utf8("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) +11)------------------TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#45"), part.p_type NOT LIKE Utf8("MEDIUM POLISHED%"), part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)])] +12)------------SubqueryAlias: __correlated_sq_1 +13)--------------Projection: supplier.s_suppkey +14)----------------Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") +15)------------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 -----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] -------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] ---------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] -----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) ---------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] +05)--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 +12)----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +21)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey], has_header=false +22)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +24)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) +26)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false +28)----------------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +30)--------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] +31)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% +33)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +34)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index a710f672a97d..19fd2375f66c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -36,45 +36,45 @@ where ); ---- logical_plan -Projection: CAST(SUM(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly ---Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice)]] -----Projection: lineitem.l_extendedprice -------Inner Join: part.p_partkey = __scalar_sq_1.l_partkey Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < __scalar_sq_1.Float64(0.2) * AVG(lineitem.l_quantity) ---------Projection: lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey -----------Inner Join: lineitem.l_partkey = part.p_partkey -------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] -------------Projection: part.p_partkey ---------------Filter: part.p_brand = Utf8("Brand#23") AND part.p_container = Utf8("MED BOX") -----------------TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8("Brand#23"), part.p_container = Utf8("MED BOX")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: CAST(Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey -------------Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] ---------------TableScan: lineitem projection=[l_partkey, l_quantity] +01)Projection: CAST(SUM(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly +02)--Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice)]] +03)----Projection: lineitem.l_extendedprice +04)------Inner Join: part.p_partkey = __scalar_sq_1.l_partkey Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < __scalar_sq_1.Float64(0.2) * AVG(lineitem.l_quantity) +05)--------Projection: lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey +06)----------Inner Join: lineitem.l_partkey = part.p_partkey +07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] +08)------------Projection: part.p_partkey +09)--------------Filter: part.p_brand = Utf8("Brand#23") AND part.p_container = Utf8("MED BOX") +10)----------------TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8("Brand#23"), part.p_container = Utf8("MED BOX")] +11)--------SubqueryAlias: __scalar_sq_1 +12)----------Projection: CAST(Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey +13)------------Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] +14)--------------TableScan: lineitem projection=[l_partkey, l_quantity] physical_plan -ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false -------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] ---------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false +01)ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false +12)----------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +14)--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +15)----------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +17)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +19)------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] +20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +24)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index d992c3a2756f..28ff0e07f363 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -51,54 +51,54 @@ order by o_orderdate; ---- logical_plan -Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST ---Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] -----LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey -------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity ---------Inner Join: orders.o_orderkey = lineitem.l_orderkey -----------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate -------------Inner Join: customer.c_custkey = orders.o_custkey ---------------TableScan: customer projection=[c_custkey, c_name] ---------------TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] -----------TableScan: lineitem projection=[l_orderkey, l_quantity] -------SubqueryAlias: __correlated_sq_1 ---------Projection: lineitem.l_orderkey -----------Filter: SUM(lineitem.l_quantity) > Decimal128(Some(30000),25,2) -------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] ---------------TableScan: lineitem projection=[l_orderkey, l_quantity] +01)Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST +02)--Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] +03)----LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey +04)------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity +05)--------Inner Join: orders.o_orderkey = lineitem.l_orderkey +06)----------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate +07)------------Inner Join: customer.c_custkey = orders.o_custkey +08)--------------TableScan: customer projection=[c_custkey, c_name] +09)--------------TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] +10)----------TableScan: lineitem projection=[l_orderkey, l_quantity] +11)------SubqueryAlias: __correlated_sq_1 +12)--------Projection: lineitem.l_orderkey +13)----------Filter: SUM(lineitem.l_quantity) > Decimal128(Some(30000),25,2) +14)------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] +15)--------------TableScan: lineitem projection=[l_orderkey, l_quantity] physical_plan -SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ---SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] -----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ---------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false -----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 -----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false +01)SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] +02)--SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] +03)----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false +22)--------------------CoalesceBatchesExec: target_batch_size=8192 +23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +24)------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false +25)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] +26)------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 +28)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] +32)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index b98969efce9b..9a49fc424eb6 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -54,34 +54,34 @@ where ); ---- logical_plan -Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue ---Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----Projection: lineitem.l_extendedprice, lineitem.l_discount -------Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) ---------Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount -----------Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") -------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG"), lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] ---------Filter: (part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) -----------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)] +01)Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue +02)--Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +03)----Projection: lineitem.l_extendedprice, lineitem.l_discount +04)------Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) +05)--------Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount +06)----------Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") +07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG"), lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] +08)--------Filter: (part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) +09)----------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)] physical_plan -ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON -----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +01)ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON +12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false +13)------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 +17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 823051be856a..088f96f32928 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -63,126 +63,126 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 -----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment -------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) ---------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name -----------Inner Join: nation.n_regionkey = region.r_regionkey -------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey ---------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost -------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey ---------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost -----------------------Inner Join: part.p_partkey = partsupp.ps_partkey -------------------------Projection: part.p_partkey, part.p_mfgr ---------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") -----------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8("%BRASS")] -------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] ---------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] -----------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -------------Projection: region.r_regionkey ---------------Filter: region.r_name = Utf8("EUROPE") -----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey -------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] ---------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost -----------------Inner Join: nation.n_regionkey = region.r_regionkey -------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey ---------------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey -------------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey ---------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] ---------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------TableScan: nation projection=[n_nationkey, n_regionkey] -------------------Projection: region.r_regionkey ---------------------Filter: region.r_name = Utf8("EUROPE") -----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] +01)Limit: skip=0, fetch=10 +02)--Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 +03)----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment +04)------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) +05)--------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name +06)----------Inner Join: nation.n_regionkey = region.r_regionkey +07)------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey +08)--------------Inner Join: supplier.s_nationkey = nation.n_nationkey +09)----------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost +10)------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +11)--------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost +12)----------------------Inner Join: part.p_partkey = partsupp.ps_partkey +13)------------------------Projection: part.p_partkey, part.p_mfgr +14)--------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") +15)----------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8("%BRASS")] +16)------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +17)--------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] +18)----------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +19)------------Projection: region.r_regionkey +20)--------------Filter: region.r_name = Utf8("EUROPE") +21)----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] +22)--------SubqueryAlias: __scalar_sq_1 +23)----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey +24)------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] +25)--------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost +26)----------------Inner Join: nation.n_regionkey = region.r_regionkey +27)------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey +28)--------------------Inner Join: supplier.s_nationkey = nation.n_nationkey +29)----------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey +30)------------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +31)--------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +32)--------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +33)----------------------TableScan: nation projection=[n_nationkey, n_regionkey] +34)------------------Projection: region.r_regionkey +35)--------------------Filter: region.r_name = Utf8("EUROPE") +36)----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 -----SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] -------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@3 as s_name, s_address@4 as s_address, s_nationkey@5 as s_nationkey, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ---------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: r_name@1 = EUROPE -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([ps_partkey@1, MIN(partsupp.ps_supplycost)@0], 4), input_partitions=4 -----------------ProjectionExec: expr=[MIN(partsupp.ps_supplycost)@1 as MIN(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 ---------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------FilterExec: r_name@1 = EUROPE -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] +04)------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 +17)--------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@3 as s_name, s_address@4 as s_address, s_nationkey@5 as s_nationkey, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +26)--------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS +29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false +31)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +33)--------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +36)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +37)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +38)----------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +40)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +42)--------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +44)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +45)--------------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------------FilterExec: r_name@1 = EUROPE +47)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +49)------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------RepartitionExec: partitioning=Hash([ps_partkey@1, MIN(partsupp.ps_supplycost)@0], 4), input_partitions=4 +51)----------------ProjectionExec: expr=[MIN(partsupp.ps_supplycost)@1 as MIN(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +52)------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +53)--------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +55)------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +56)--------------------------CoalesceBatchesExec: target_batch_size=8192 +57)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +58)------------------------------CoalesceBatchesExec: target_batch_size=8192 +59)--------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +60)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +61)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +62)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +63)----------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +64)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +65)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +66)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +67)------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +68)--------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +69)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +70)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +71)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +72)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +73)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +74)----------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +75)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +76)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +77)------------------------------CoalesceBatchesExec: target_batch_size=8192 +78)--------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +79)----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +80)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +81)--------------------------------------FilterExec: r_name@1 = EUROPE +82)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +83)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index fad77353651c..4b01f32a94ca 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -56,77 +56,77 @@ order by s_name; ---- logical_plan -Sort: supplier.s_name ASC NULLS LAST ---Projection: supplier.s_name, supplier.s_address -----LeftSemi Join: supplier.s_suppkey = __correlated_sq_1.ps_suppkey -------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address ---------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] -----------Projection: nation.n_nationkey -------------Filter: nation.n_name = Utf8("CANADA") ---------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("CANADA")] -------SubqueryAlias: __correlated_sq_1 ---------Projection: partsupp.ps_suppkey -----------Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * SUM(lineitem.l_quantity) -------------LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey ---------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] ---------------SubqueryAlias: __correlated_sq_2 -----------------Projection: part.p_partkey -------------------Filter: part.p_name LIKE Utf8("forest%") ---------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("forest%")] -------------SubqueryAlias: __scalar_sq_3 ---------------Projection: Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey -----------------Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] -------------------Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity ---------------------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") -----------------------TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131")] +01)Sort: supplier.s_name ASC NULLS LAST +02)--Projection: supplier.s_name, supplier.s_address +03)----LeftSemi Join: supplier.s_suppkey = __correlated_sq_1.ps_suppkey +04)------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address +05)--------Inner Join: supplier.s_nationkey = nation.n_nationkey +06)----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] +07)----------Projection: nation.n_nationkey +08)------------Filter: nation.n_name = Utf8("CANADA") +09)--------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("CANADA")] +10)------SubqueryAlias: __correlated_sq_1 +11)--------Projection: partsupp.ps_suppkey +12)----------Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * SUM(lineitem.l_quantity) +13)------------LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey +14)--------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] +15)--------------SubqueryAlias: __correlated_sq_2 +16)----------------Projection: part.p_partkey +17)------------------Filter: part.p_name LIKE Utf8("forest%") +18)--------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("forest%")] +19)------------SubqueryAlias: __scalar_sq_3 +20)--------------Projection: Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey +21)----------------Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] +22)------------------Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity +23)--------------------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") +24)----------------------TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131")] physical_plan -SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ---SortExec: expr=[s_name@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 -------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------FilterExec: n_name@1 = CANADA ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------FilterExec: p_name@1 LIKE forest% -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] ---------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 ---------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false +01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] +02)--SortExec: expr=[s_name@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +15)--------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +16)----------------------CoalesceBatchesExec: target_batch_size=8192 +17)------------------------FilterExec: n_name@1 = CANADA +18)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +20)--------CoalesceBatchesExec: target_batch_size=8192 +21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +22)------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +24)----------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +26)--------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +28)------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +30)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false +31)------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +33)----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +34)------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------------------FilterExec: p_name@1 LIKE forest% +36)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +37)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +38)----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +40)--------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +42)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +43)--------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] +44)----------------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 +46)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index a2246381e601..944a68abd327 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -58,93 +58,93 @@ order by s_name; ---- logical_plan -Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST ---Projection: supplier.s_name, COUNT(*) AS numwait -----Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -------Projection: supplier.s_name ---------LeftAnti Join: l1.l_orderkey = __correlated_sq_2.l_orderkey Filter: __correlated_sq_2.l_suppkey != l1.l_suppkey -----------LeftSemi Join: l1.l_orderkey = __correlated_sq_1.l_orderkey Filter: __correlated_sq_1.l_suppkey != l1.l_suppkey -------------Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey ---------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey -------------------Inner Join: l1.l_orderkey = orders.o_orderkey ---------------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey -----------------------Inner Join: supplier.s_suppkey = l1.l_suppkey -------------------------TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] -------------------------SubqueryAlias: l1 ---------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey -----------------------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] ---------------------Projection: orders.o_orderkey -----------------------Filter: orders.o_orderstatus = Utf8("F") -------------------------TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8("F")] -----------------Projection: nation.n_nationkey -------------------Filter: nation.n_name = Utf8("SAUDI ARABIA") ---------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("SAUDI ARABIA")] -------------SubqueryAlias: __correlated_sq_1 ---------------SubqueryAlias: l2 -----------------TableScan: lineitem projection=[l_orderkey, l_suppkey] -----------SubqueryAlias: __correlated_sq_2 -------------SubqueryAlias: l3 ---------------Projection: lineitem.l_orderkey, lineitem.l_suppkey -----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +01)Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST +02)--Projection: supplier.s_name, COUNT(*) AS numwait +03)----Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------Projection: supplier.s_name +05)--------LeftAnti Join: l1.l_orderkey = __correlated_sq_2.l_orderkey Filter: __correlated_sq_2.l_suppkey != l1.l_suppkey +06)----------LeftSemi Join: l1.l_orderkey = __correlated_sq_1.l_orderkey Filter: __correlated_sq_1.l_suppkey != l1.l_suppkey +07)------------Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey +08)--------------Inner Join: supplier.s_nationkey = nation.n_nationkey +09)----------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey +10)------------------Inner Join: l1.l_orderkey = orders.o_orderkey +11)--------------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey +12)----------------------Inner Join: supplier.s_suppkey = l1.l_suppkey +13)------------------------TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] +14)------------------------SubqueryAlias: l1 +15)--------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey +16)----------------------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate +17)------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +18)--------------------Projection: orders.o_orderkey +19)----------------------Filter: orders.o_orderstatus = Utf8("F") +20)------------------------TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8("F")] +21)----------------Projection: nation.n_nationkey +22)------------------Filter: nation.n_name = Utf8("SAUDI ARABIA") +23)--------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("SAUDI ARABIA")] +24)------------SubqueryAlias: __correlated_sq_1 +25)--------------SubqueryAlias: l2 +26)----------------TableScan: lineitem projection=[l_orderkey, l_suppkey] +27)----------SubqueryAlias: __correlated_sq_2 +28)------------SubqueryAlias: l3 +29)--------------Projection: lineitem.l_orderkey, lineitem.l_suppkey +30)----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate +31)------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan -SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ---SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST] -----ProjectionExec: expr=[s_name@0 as s_name, COUNT(*)@1 as numwait] -------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(*)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ---------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 ---------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------FilterExec: o_orderstatus@1 = F -------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------FilterExec: n_name@1 = SAUDI ARABIA -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +01)SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] +02)--SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[s_name@0 as s_name, COUNT(*)@1 as numwait] +04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false +28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +30)--------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +33)--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +36)------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] +37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------------------------------FilterExec: o_orderstatus@1 = F +39)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false +40)------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +42)----------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------------FilterExec: n_name@1 = SAUDI ARABIA +45)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +49)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false +50)------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +52)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +53)------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +55)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index fea82b9022a2..73b3c1602565 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -56,52 +56,52 @@ order by cntrycode; ---- logical_plan -Sort: custsale.cntrycode ASC NULLS LAST ---Projection: custsale.cntrycode, COUNT(*) AS numcust, SUM(custsale.c_acctbal) AS totacctbal -----Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[COUNT(Int64(1)) AS COUNT(*), SUM(custsale.c_acctbal)]] -------SubqueryAlias: custsale ---------Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal -----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.AVG(customer.c_acctbal) -------------Projection: customer.c_phone, customer.c_acctbal ---------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey -----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) -------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])] -----------------SubqueryAlias: __correlated_sq_1 -------------------TableScan: orders projection=[o_custkey] -------------SubqueryAlias: __scalar_sq_2 ---------------Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] -----------------Projection: customer.c_acctbal -------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) ---------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2) AS customer.c_acctbal > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]), customer.c_acctbal > Decimal128(Some(0),15,2)] +01)Sort: custsale.cntrycode ASC NULLS LAST +02)--Projection: custsale.cntrycode, COUNT(*) AS numcust, SUM(custsale.c_acctbal) AS totacctbal +03)----Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[COUNT(Int64(1)) AS COUNT(*), SUM(custsale.c_acctbal)]] +04)------SubqueryAlias: custsale +05)--------Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal +06)----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.AVG(customer.c_acctbal) +07)------------Projection: customer.c_phone, customer.c_acctbal +08)--------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey +09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) +10)------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])] +11)----------------SubqueryAlias: __correlated_sq_1 +12)------------------TableScan: orders projection=[o_custkey] +13)------------SubqueryAlias: __scalar_sq_2 +14)--------------Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] +15)----------------Projection: customer.c_acctbal +16)------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) +17)--------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2) AS customer.c_acctbal > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]), customer.c_acctbal > Decimal128(Some(0),15,2)] physical_plan -SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ---SortExec: expr=[cntrycode@0 ASC NULLS LAST] -----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(*)@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] -------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] ---------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] -----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 ---------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], has_header=false -------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] ---------------------CoalescePartitionsExec -----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] -------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +01)SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] +02)--SortExec: expr=[cntrycode@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(*)@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] +04)------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] +08)--------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] +09)----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) +16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false +18)----------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +20)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], has_header=false +21)------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] +22)--------------------CoalescePartitionsExec +23)----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] +24)------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] +25)--------------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) +27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false query TIR diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index caedefeb051e..fe7816715632 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -42,55 +42,55 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=10 -----Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority -------Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] ---------Projection: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount -----------Inner Join: orders.o_orderkey = lineitem.l_orderkey -------------Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority ---------------Inner Join: customer.c_custkey = orders.o_custkey -----------------Projection: customer.c_custkey -------------------Filter: customer.c_mktsegment = Utf8("BUILDING") ---------------------TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8("BUILDING")] -----------------Filter: orders.o_orderdate < Date32("9204") -------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("9204")] -------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount ---------------Filter: lineitem.l_shipdate > Date32("9204") -----------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] +01)Limit: skip=0, fetch=10 +02)--Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=10 +03)----Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority +04)------Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +05)--------Projection: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +06)----------Inner Join: orders.o_orderkey = lineitem.l_orderkey +07)------------Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority +08)--------------Inner Join: customer.c_custkey = orders.o_custkey +09)----------------Projection: customer.c_custkey +10)------------------Filter: customer.c_mktsegment = Utf8("BUILDING") +11)--------------------TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8("BUILDING")] +12)----------------Filter: orders.o_orderdate < Date32("9204") +13)------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("9204")] +14)------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +15)--------------Filter: lineitem.l_shipdate > Date32("9204") +16)----------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 -----SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] -------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] ---------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: c_mktsegment@1 = BUILDING ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------FilterExec: o_orderdate@2 < 9204 -------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: l_shipdate@3 > 9204 -------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] +05)--------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +17)--------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------FilterExec: c_mktsegment@1 = BUILDING +20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +22)----------------------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +24)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------------------FilterExec: o_orderdate@2 < 9204 +26)------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false +27)--------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +30)--------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------FilterExec: l_shipdate@3 > 9204 +32)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index 1c35887c0665..eb069cdce081 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -40,40 +40,40 @@ order by o_orderpriority; ---- logical_plan -Sort: orders.o_orderpriority ASC NULLS LAST ---Projection: orders.o_orderpriority, COUNT(*) AS order_count -----Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -------Projection: orders.o_orderpriority ---------LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey -----------Projection: orders.o_orderkey, orders.o_orderpriority -------------Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") ---------------TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("8582"), orders.o_orderdate < Date32("8674")] -----------SubqueryAlias: __correlated_sq_1 -------------Projection: lineitem.l_orderkey ---------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -----------------TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +01)Sort: orders.o_orderpriority ASC NULLS LAST +02)--Projection: orders.o_orderpriority, COUNT(*) AS order_count +03)----Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------Projection: orders.o_orderpriority +05)--------LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey +06)----------Projection: orders.o_orderkey, orders.o_orderpriority +07)------------Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") +08)--------------TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("8582"), orders.o_orderdate < Date32("8674")] +09)----------SubqueryAlias: __correlated_sq_1 +10)------------Projection: lineitem.l_orderkey +11)--------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate +12)----------------TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan -SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] ---SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] -----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(*)@1 as order_count] -------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false +01)SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] +02)--SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(*)@1 as order_count] +04)------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +12)----------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 +15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] +19)------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 +21)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index 80dfc686eb49..188573069269 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -43,83 +43,83 @@ order by revenue desc; ---- logical_plan -Sort: revenue DESC NULLS FIRST ---Projection: nation.n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue -----Aggregate: groupBy=[[nation.n_name]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name ---------Inner Join: nation.n_regionkey = region.r_regionkey -----------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey -------------Inner Join: supplier.s_nationkey = nation.n_nationkey ---------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey -----------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey -------------------Projection: customer.c_nationkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount ---------------------Inner Join: orders.o_orderkey = lineitem.l_orderkey -----------------------Projection: customer.c_nationkey, orders.o_orderkey -------------------------Inner Join: customer.c_custkey = orders.o_custkey ---------------------------TableScan: customer projection=[c_custkey, c_nationkey] ---------------------------Projection: orders.o_orderkey, orders.o_custkey -----------------------------Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") -------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8766"), orders.o_orderdate < Date32("9131")] -----------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] -------------------TableScan: supplier projection=[s_suppkey, s_nationkey] ---------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -----------Projection: region.r_regionkey -------------Filter: region.r_name = Utf8("ASIA") ---------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("ASIA")] +01)Sort: revenue DESC NULLS FIRST +02)--Projection: nation.n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue +03)----Aggregate: groupBy=[[nation.n_name]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +04)------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name +05)--------Inner Join: nation.n_regionkey = region.r_regionkey +06)----------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey +07)------------Inner Join: supplier.s_nationkey = nation.n_nationkey +08)--------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey +09)----------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey +10)------------------Projection: customer.c_nationkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +11)--------------------Inner Join: orders.o_orderkey = lineitem.l_orderkey +12)----------------------Projection: customer.c_nationkey, orders.o_orderkey +13)------------------------Inner Join: customer.c_custkey = orders.o_custkey +14)--------------------------TableScan: customer projection=[c_custkey, c_nationkey] +15)--------------------------Projection: orders.o_orderkey, orders.o_custkey +16)----------------------------Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") +17)------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8766"), orders.o_orderdate < Date32("9131")] +18)----------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] +19)------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +20)--------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +21)----------Projection: region.r_regionkey +22)------------Filter: region.r_name = Utf8("ASIA") +23)--------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("ASIA")] physical_plan -SortPreservingMergeExec: [revenue@1 DESC] ---SortExec: expr=[revenue@1 DESC] -----ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] -------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 -------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: r_name@1 = ASIA -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +01)SortPreservingMergeExec: [revenue@1 DESC] +02)--SortExec: expr=[revenue@1 DESC] +03)----ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] +04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +32)------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 +35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +36)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +38)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false +39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +43)--------------------------CoalesceBatchesExec: target_batch_size=8192 +44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +45)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +49)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +50)------------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------------FilterExec: r_name@1 = ASIA +52)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +53)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/q6.slt.part index 8e53be297db0..e54b3c1ccd03 100644 --- a/datafusion/sqllogictest/test_files/tpch/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q6.slt.part @@ -28,20 +28,20 @@ where and l_quantity < 24; ---- logical_plan -Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue ---Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] -----Projection: lineitem.l_extendedprice, lineitem.l_discount -------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) ---------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131"), lineitem.l_discount >= Decimal128(Some(5),15,2), lineitem.l_discount <= Decimal128(Some(7),15,2), lineitem.l_quantity < Decimal128(Some(2400),15,2)] +01)Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue +02)--Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] +03)----Projection: lineitem.l_extendedprice, lineitem.l_discount +04)------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) +05)--------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131"), lineitem.l_discount >= Decimal128(Some(5),15,2), lineitem.l_discount <= Decimal128(Some(7),15,2), lineitem.l_quantity < Decimal128(Some(2400),15,2)] physical_plan -ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] ---------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------CoalesceBatchesExec: target_batch_size=8192 -------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 ---------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false +01)ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] +05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 +08)--------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 60f0a5879937..f699c3ee6734 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -58,86 +58,86 @@ order by l_year; ---- logical_plan -Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, shipping.l_year ASC NULLS LAST ---Projection: shipping.supp_nation, shipping.cust_nation, shipping.l_year, SUM(shipping.volume) AS revenue -----Aggregate: groupBy=[[shipping.supp_nation, shipping.cust_nation, shipping.l_year]], aggr=[[SUM(shipping.volume)]] -------SubqueryAlias: shipping ---------Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, date_part(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume -----------Inner Join: customer.c_nationkey = n2.n_nationkey Filter: n1.n_name = Utf8("FRANCE") AND n2.n_name = Utf8("GERMANY") OR n1.n_name = Utf8("GERMANY") AND n2.n_name = Utf8("FRANCE") -------------Projection: lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey, n1.n_name ---------------Inner Join: supplier.s_nationkey = n1.n_nationkey -----------------Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey -------------------Inner Join: orders.o_custkey = customer.c_custkey ---------------------Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, orders.o_custkey -----------------------Inner Join: lineitem.l_orderkey = orders.o_orderkey -------------------------Projection: supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate ---------------------------Inner Join: supplier.s_suppkey = lineitem.l_suppkey -----------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------------Filter: lineitem.l_shipdate >= Date32("9131") AND lineitem.l_shipdate <= Date32("9861") -------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9131"), lineitem.l_shipdate <= Date32("9861")] -------------------------TableScan: orders projection=[o_orderkey, o_custkey] ---------------------TableScan: customer projection=[c_custkey, c_nationkey] -----------------SubqueryAlias: n1 -------------------Filter: nation.n_name = Utf8("FRANCE") OR nation.n_name = Utf8("GERMANY") ---------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("FRANCE") OR nation.n_name = Utf8("GERMANY")] -------------SubqueryAlias: n2 ---------------Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE") -----------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE")] +01)Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST, shipping.l_year ASC NULLS LAST +02)--Projection: shipping.supp_nation, shipping.cust_nation, shipping.l_year, SUM(shipping.volume) AS revenue +03)----Aggregate: groupBy=[[shipping.supp_nation, shipping.cust_nation, shipping.l_year]], aggr=[[SUM(shipping.volume)]] +04)------SubqueryAlias: shipping +05)--------Projection: n1.n_name AS supp_nation, n2.n_name AS cust_nation, date_part(Utf8("YEAR"), lineitem.l_shipdate) AS l_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume +06)----------Inner Join: customer.c_nationkey = n2.n_nationkey Filter: n1.n_name = Utf8("FRANCE") AND n2.n_name = Utf8("GERMANY") OR n1.n_name = Utf8("GERMANY") AND n2.n_name = Utf8("FRANCE") +07)------------Projection: lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey, n1.n_name +08)--------------Inner Join: supplier.s_nationkey = n1.n_nationkey +09)----------------Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, customer.c_nationkey +10)------------------Inner Join: orders.o_custkey = customer.c_custkey +11)--------------------Projection: supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, orders.o_custkey +12)----------------------Inner Join: lineitem.l_orderkey = orders.o_orderkey +13)------------------------Projection: supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate +14)--------------------------Inner Join: supplier.s_suppkey = lineitem.l_suppkey +15)----------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +16)----------------------------Filter: lineitem.l_shipdate >= Date32("9131") AND lineitem.l_shipdate <= Date32("9861") +17)------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9131"), lineitem.l_shipdate <= Date32("9861")] +18)------------------------TableScan: orders projection=[o_orderkey, o_custkey] +19)--------------------TableScan: customer projection=[c_custkey, c_nationkey] +20)----------------SubqueryAlias: n1 +21)------------------Filter: nation.n_name = Utf8("FRANCE") OR nation.n_name = Utf8("GERMANY") +22)--------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("FRANCE") OR nation.n_name = Utf8("GERMANY")] +23)------------SubqueryAlias: n2 +24)--------------Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE") +25)----------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE")] physical_plan -SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] ---SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] -----ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, SUM(shipping.volume)@3 as revenue] -------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] ---------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 -------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] +02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] +03)----ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, SUM(shipping.volume)@3 as revenue] +04)------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] +08)--------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 +35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +38)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], has_header=false +39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +41)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +45)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +47)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +49)--------------------CoalesceBatchesExec: target_batch_size=8192 +50)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +51)------------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +53)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +54)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 999710df7609..ec89bd0c0806 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -56,112 +56,112 @@ order by o_year; ---- logical_plan -Sort: all_nations.o_year ASC NULLS LAST ---Projection: all_nations.o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END) AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume) AS Decimal128(12, 2)) AS Decimal128(15, 2)) AS mkt_share -----Aggregate: groupBy=[[all_nations.o_year]], aggr=[[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]] -------SubqueryAlias: all_nations ---------Projection: date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume, n2.n_name AS nation -----------Inner Join: n1.n_regionkey = region.r_regionkey -------------Projection: lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, n1.n_regionkey, n2.n_name ---------------Inner Join: supplier.s_nationkey = n2.n_nationkey -----------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, n1.n_regionkey -------------------Inner Join: customer.c_nationkey = n1.n_nationkey ---------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, customer.c_nationkey -----------------------Inner Join: orders.o_custkey = customer.c_custkey -------------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_custkey, orders.o_orderdate ---------------------------Inner Join: lineitem.l_orderkey = orders.o_orderkey -----------------------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey -------------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey ---------------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -----------------------------------Inner Join: part.p_partkey = lineitem.l_partkey -------------------------------------Projection: part.p_partkey ---------------------------------------Filter: part.p_type = Utf8("ECONOMY ANODIZED STEEL") -----------------------------------------TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8("ECONOMY ANODIZED STEEL")] -------------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] ---------------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------------Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") -------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("9131"), orders.o_orderdate <= Date32("9861")] -------------------------TableScan: customer projection=[c_custkey, c_nationkey] ---------------------SubqueryAlias: n1 -----------------------TableScan: nation projection=[n_nationkey, n_regionkey] -----------------SubqueryAlias: n2 -------------------TableScan: nation projection=[n_nationkey, n_name] -------------Projection: region.r_regionkey ---------------Filter: region.r_name = Utf8("AMERICA") -----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("AMERICA")] +01)Sort: all_nations.o_year ASC NULLS LAST +02)--Projection: all_nations.o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END) AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume) AS Decimal128(12, 2)) AS Decimal128(15, 2)) AS mkt_share +03)----Aggregate: groupBy=[[all_nations.o_year]], aggr=[[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]] +04)------SubqueryAlias: all_nations +05)--------Projection: date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume, n2.n_name AS nation +06)----------Inner Join: n1.n_regionkey = region.r_regionkey +07)------------Projection: lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, n1.n_regionkey, n2.n_name +08)--------------Inner Join: supplier.s_nationkey = n2.n_nationkey +09)----------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, n1.n_regionkey +10)------------------Inner Join: customer.c_nationkey = n1.n_nationkey +11)--------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, customer.c_nationkey +12)----------------------Inner Join: orders.o_custkey = customer.c_custkey +13)------------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_custkey, orders.o_orderdate +14)--------------------------Inner Join: lineitem.l_orderkey = orders.o_orderkey +15)----------------------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey +16)------------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey +17)--------------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +18)----------------------------------Inner Join: part.p_partkey = lineitem.l_partkey +19)------------------------------------Projection: part.p_partkey +20)--------------------------------------Filter: part.p_type = Utf8("ECONOMY ANODIZED STEEL") +21)----------------------------------------TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8("ECONOMY ANODIZED STEEL")] +22)------------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] +23)--------------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +24)----------------------------Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") +25)------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("9131"), orders.o_orderdate <= Date32("9861")] +26)------------------------TableScan: customer projection=[c_custkey, c_nationkey] +27)--------------------SubqueryAlias: n1 +28)----------------------TableScan: nation projection=[n_nationkey, n_regionkey] +29)----------------SubqueryAlias: n2 +30)------------------TableScan: nation projection=[n_nationkey, n_name] +31)------------Projection: region.r_regionkey +32)--------------Filter: region.r_name = Utf8("AMERICA") +33)----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("AMERICA")] physical_plan -SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] ---SortExec: expr=[o_year@0 ASC NULLS LAST] -----ProjectionExec: expr=[o_year@0 as o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume)@2 AS Decimal128(12, 2)) AS Decimal128(15, 2)) as mkt_share] -------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] ---------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] ---------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ---------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL -------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false ---------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 -------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: r_name@1 = AMERICA -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +01)SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] +02)--SortExec: expr=[o_year@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[o_year@0 as o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume)@2 AS Decimal128(12, 2)) AS Decimal128(15, 2)) as mkt_share] +04)------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] +08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] +31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +33)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] +35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +37)------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +38)--------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL +40)------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)--------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +44)------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 +53)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +62)----------------------------CoalesceBatchesExec: target_batch_size=8192 +63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +65)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +66)--------------------CoalesceBatchesExec: target_batch_size=8192 +67)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +68)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +69)--------------------------CoalesceBatchesExec: target_batch_size=8192 +70)----------------------------FilterExec: r_name@1 = AMERICA +71)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +72)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 19abb56dfcb5..61ed162aa712 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -52,81 +52,81 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST, fetch=10 -----Projection: profit.nation, profit.o_year, SUM(profit.amount) AS sum_profit -------Aggregate: groupBy=[[profit.nation, profit.o_year]], aggr=[[SUM(profit.amount)]] ---------SubqueryAlias: profit -----------Projection: nation.n_name AS nation, date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) - partsupp.ps_supplycost * lineitem.l_quantity AS amount -------------Inner Join: supplier.s_nationkey = nation.n_nationkey ---------------Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost, orders.o_orderdate -----------------Inner Join: lineitem.l_orderkey = orders.o_orderkey -------------------Projection: lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost ---------------------Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey -----------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey -------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey ---------------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount -----------------------------Inner Join: part.p_partkey = lineitem.l_partkey -------------------------------Projection: part.p_partkey ---------------------------------Filter: part.p_name LIKE Utf8("%green%") -----------------------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("%green%")] -------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] ---------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] -------------------TableScan: orders projection=[o_orderkey, o_orderdate] ---------------TableScan: nation projection=[n_nationkey, n_name] +01)Limit: skip=0, fetch=10 +02)--Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST, fetch=10 +03)----Projection: profit.nation, profit.o_year, SUM(profit.amount) AS sum_profit +04)------Aggregate: groupBy=[[profit.nation, profit.o_year]], aggr=[[SUM(profit.amount)]] +05)--------SubqueryAlias: profit +06)----------Projection: nation.n_name AS nation, date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) - partsupp.ps_supplycost * lineitem.l_quantity AS amount +07)------------Inner Join: supplier.s_nationkey = nation.n_nationkey +08)--------------Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost, orders.o_orderdate +09)----------------Inner Join: lineitem.l_orderkey = orders.o_orderkey +10)------------------Projection: lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost +11)--------------------Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey +12)----------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey +13)------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey +14)--------------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount +15)----------------------------Inner Join: part.p_partkey = lineitem.l_partkey +16)------------------------------Projection: part.p_partkey +17)--------------------------------Filter: part.p_name LIKE Utf8("%green%") +18)----------------------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("%green%")] +19)------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] +20)--------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +21)----------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +22)------------------TableScan: orders projection=[o_orderkey, o_orderdate] +23)--------------TableScan: nation projection=[n_nationkey, n_name] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] -------ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] ---------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------FilterExec: p_name@1 LIKE %green% -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -----------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] +04)------ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] +05)--------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] +09)----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] +28)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +30)----------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------------------------------FilterExec: p_name@1 LIKE %green% +33)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +34)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +35)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +37)----------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +38)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +40)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +42)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +44)------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +45)------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +47)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index b87580a7bdd0..ba2563fd5a4d 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -196,23 +196,23 @@ query TT EXPLAIN SELECT name FROM t1 UNION (SELECT name from t2 UNION SELECT name || '_new' from t2) ---- logical_plan -Aggregate: groupBy=[[t1.name]], aggr=[[]] ---Union -----TableScan: t1 projection=[name] -----TableScan: t2 projection=[name] -----Projection: t2.name || Utf8("_new") AS name -------TableScan: t2 projection=[name] +01)Aggregate: groupBy=[[t1.name]], aggr=[[]] +02)--Union +03)----TableScan: t1 projection=[name] +04)----TableScan: t2 projection=[name] +05)----Projection: t2.name || Utf8("_new") AS name +06)------TableScan: t2 projection=[name] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 ---------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -----------UnionExec -------------MemoryExec: partitions=1, partition_sizes=[1] -------------MemoryExec: partitions=1, partition_sizes=[1] -------------ProjectionExec: expr=[name@0 || _new as name] ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 +05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +06)----------UnionExec +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------------MemoryExec: partitions=1, partition_sizes=[1] +09)------------ProjectionExec: expr=[name@0 || _new as name] +10)--------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -233,17 +233,17 @@ query TT EXPLAIN SELECT name FROM t1 UNION ALL (SELECT name from t2 UNION ALL SELECT name || '_new' from t2) ---- logical_plan -Union ---TableScan: t1 projection=[name] ---TableScan: t2 projection=[name] ---Projection: t2.name || Utf8("_new") AS name -----TableScan: t2 projection=[name] +01)Union +02)--TableScan: t1 projection=[name] +03)--TableScan: t2 projection=[name] +04)--Projection: t2.name || Utf8("_new") AS name +05)----TableScan: t2 projection=[name] physical_plan -UnionExec ---MemoryExec: partitions=1, partition_sizes=[1] ---MemoryExec: partitions=1, partition_sizes=[1] ---ProjectionExec: expr=[name@0 || _new as name] -----MemoryExec: partitions=1, partition_sizes=[1] +01)UnionExec +02)--MemoryExec: partitions=1, partition_sizes=[1] +03)--MemoryExec: partitions=1, partition_sizes=[1] +04)--ProjectionExec: expr=[name@0 || _new as name] +05)----MemoryExec: partitions=1, partition_sizes=[1] # Make sure to choose a small batch size to introduce parallelism to the plan. statement ok @@ -265,47 +265,47 @@ UNION ALL ) ---- logical_plan -Union ---LeftAnti Join: t1.id = CAST(t2.id AS Int32), t1.name = t2.name -----Aggregate: groupBy=[[t1.id, t1.name]], aggr=[[]] -------TableScan: t1 projection=[id, name] -----TableScan: t2 projection=[id, name] ---Projection: CAST(t2.id AS Int32) AS id, t2.name -----LeftAnti Join: CAST(t2.id AS Int32) = t1.id, t2.name = t1.name -------Aggregate: groupBy=[[t2.id, t2.name]], aggr=[[]] ---------TableScan: t2 projection=[id, name] -------TableScan: t1 projection=[id, name] +01)Union +02)--LeftAnti Join: t1.id = CAST(t2.id AS Int32), t1.name = t2.name +03)----Aggregate: groupBy=[[t1.id, t1.name]], aggr=[[]] +04)------TableScan: t1 projection=[id, name] +05)----TableScan: t2 projection=[id, name] +06)--Projection: CAST(t2.id AS Int32) AS id, t2.name +07)----LeftAnti Join: CAST(t2.id AS Int32) = t1.id, t2.name = t1.name +08)------Aggregate: groupBy=[[t2.id, t2.name]], aggr=[[]] +09)--------TableScan: t2 projection=[id, name] +10)------TableScan: t1 projection=[id, name] physical_plan -UnionExec ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] -------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] ---------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)UnionExec +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] +04)------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------CoalesceBatchesExec: target_batch_size=2 +11)--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +12)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------MemoryExec: partitions=1, partition_sizes=[1] +15)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +16)----CoalesceBatchesExec: target_batch_size=2 +17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +18)--------CoalesceBatchesExec: target_batch_size=2 +19)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +20)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +21)--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +22)----------------CoalesceBatchesExec: target_batch_size=2 +23)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +24)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +25)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)------------------------MemoryExec: partitions=1, partition_sizes=[1] +27)--------CoalesceBatchesExec: target_batch_size=2 +28)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +29)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -340,66 +340,66 @@ UNION ALL ) ---- logical_plan -Union ---LeftAnti Join: t1.name = t2.name -----Aggregate: groupBy=[[t1.name]], aggr=[[]] -------TableScan: t1 projection=[name] -----TableScan: t2 projection=[name] ---LeftAnti Join: t2.name = t1.name -----Aggregate: groupBy=[[t2.name]], aggr=[[]] -------TableScan: t2 projection=[name] -----TableScan: t1 projection=[name] +01)Union +02)--LeftAnti Join: t1.name = t2.name +03)----Aggregate: groupBy=[[t1.name]], aggr=[[]] +04)------TableScan: t1 projection=[name] +05)----TableScan: t2 projection=[name] +06)--LeftAnti Join: t2.name = t1.name +07)----Aggregate: groupBy=[[t2.name]], aggr=[[]] +08)------TableScan: t2 projection=[name] +09)----TableScan: t1 projection=[name] physical_plan -InterleaveExec ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] -------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] -------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)InterleaveExec +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] +04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------CoalesceBatchesExec: target_batch_size=2 +11)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +12)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------MemoryExec: partitions=1, partition_sizes=[1] +14)--CoalesceBatchesExec: target_batch_size=2 +15)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] +16)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +17)--------CoalesceBatchesExec: target_batch_size=2 +18)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +19)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +20)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)----------------MemoryExec: partitions=1, partition_sizes=[1] +22)------CoalesceBatchesExec: target_batch_size=2 +23)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +24)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)------------MemoryExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT explain SELECT c1, c9 FROM aggregate_test_100 UNION ALL SELECT c1, c3 FROM aggregate_test_100 ORDER BY c9 DESC LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 -----Union -------Projection: aggregate_test_100.c1, CAST(aggregate_test_100.c9 AS Int64) AS c9 ---------TableScan: aggregate_test_100 projection=[c1, c9] -------Projection: aggregate_test_100.c1, CAST(aggregate_test_100.c3 AS Int64) AS c9 ---------TableScan: aggregate_test_100 projection=[c1, c3] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 +03)----Union +04)------Projection: aggregate_test_100.c1, CAST(aggregate_test_100.c9 AS Int64) AS c9 +05)--------TableScan: aggregate_test_100 projection=[c1, c9] +06)------Projection: aggregate_test_100.c1, CAST(aggregate_test_100.c3 AS Int64) AS c9 +07)--------TableScan: aggregate_test_100 projection=[c1, c3] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [c9@1 DESC], fetch=5 -----UnionExec -------SortExec: expr=[c9@1 DESC] ---------ProjectionExec: expr=[c1@0 as c1, CAST(c9@1 AS Int64) as c9] -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true -------SortExec: expr=[c9@1 DESC] ---------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Int64) as c9] -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [c9@1 DESC], fetch=5 +03)----UnionExec +04)------SortExec: expr=[c9@1 DESC] +05)--------ProjectionExec: expr=[c1@0 as c1, CAST(c9@1 AS Int64) as c9] +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +08)------SortExec: expr=[c9@1 DESC] +09)--------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Int64) as c9] +10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true query TI SELECT c1, c9 FROM aggregate_test_100 UNION ALL SELECT c1, c3 FROM aggregate_test_100 ORDER BY c9 DESC LIMIT 5 @@ -420,29 +420,29 @@ SELECT count(*) FROM ( ) GROUP BY name ---- logical_plan -Projection: COUNT(*) ---Aggregate: groupBy=[[t1.name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----Union -------Aggregate: groupBy=[[t1.name]], aggr=[[]] ---------TableScan: t1 projection=[name] -------Aggregate: groupBy=[[t2.name]], aggr=[[]] ---------TableScan: t2 projection=[name] +01)Projection: COUNT(*) +02)--Aggregate: groupBy=[[t1.name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----Union +04)------Aggregate: groupBy=[[t1.name]], aggr=[[]] +05)--------TableScan: t1 projection=[name] +06)------Aggregate: groupBy=[[t2.name]], aggr=[[]] +07)--------TableScan: t2 projection=[name] physical_plan -ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] ---AggregateExec: mode=SinglePartitioned, gby=[name@0 as name], aggr=[COUNT(*)] -----InterleaveExec -------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] +02)--AggregateExec: mode=SinglePartitioned, gby=[name@0 as name], aggr=[COUNT(*)] +03)----InterleaveExec +04)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +11)--------CoalesceBatchesExec: target_batch_size=2 +12)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +13)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------MemoryExec: partitions=1, partition_sizes=[1] ######## @@ -513,17 +513,17 @@ UNION ALL ORDER BY c1 ---- logical_plan -Sort: t1.c1 ASC NULLS LAST ---Union -----TableScan: t1 projection=[c1] -----Projection: t2.c1a AS c1 -------TableScan: t2 projection=[c1a] +01)Sort: t1.c1 ASC NULLS LAST +02)--Union +03)----TableScan: t1 projection=[c1] +04)----Projection: t2.c1a AS c1 +05)------TableScan: t2 projection=[c1a] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ---UnionExec -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], has_header=true -----ProjectionExec: expr=[c1a@0 as c1] -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +02)--UnionExec +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], has_header=true +04)----ProjectionExec: expr=[c1a@0 as c1] +05)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], has_header=true statement ok drop table t1 @@ -536,26 +536,26 @@ query TT explain select 1 a group by a union all select 2 b union all select 3 c ---- logical_plan -Union ---Projection: Int64(1) AS a -----Aggregate: groupBy=[[Int64(1)]], aggr=[[]] -------EmptyRelation ---Projection: Int64(2) AS a -----EmptyRelation ---Projection: Int64(3) AS a -----EmptyRelation +01)Union +02)--Projection: Int64(1) AS a +03)----Aggregate: groupBy=[[Int64(1)]], aggr=[[]] +04)------EmptyRelation +05)--Projection: Int64(2) AS a +06)----EmptyRelation +07)--Projection: Int64(3) AS a +08)----EmptyRelation physical_plan -UnionExec ---ProjectionExec: expr=[Int64(1)@0 as a] -----AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1)], aggr=[], ordering_mode=Sorted -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=1 -----------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted -------------PlaceholderRowExec ---ProjectionExec: expr=[2 as a] -----PlaceholderRowExec ---ProjectionExec: expr=[3 as a] -----PlaceholderRowExec +01)UnionExec +02)--ProjectionExec: expr=[Int64(1)@0 as a] +03)----AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1)], aggr=[], ordering_mode=Sorted +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted +07)------------PlaceholderRowExec +08)--ProjectionExec: expr=[2 as a] +09)----PlaceholderRowExec +10)--ProjectionExec: expr=[3 as a] +11)----PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT @@ -564,26 +564,26 @@ union all select x, y from (select 1 as x , max(10) as y) b ---- logical_plan -Union ---Projection: COUNT(*) AS count, a.n -----Aggregate: groupBy=[[a.n]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -------SubqueryAlias: a ---------Projection: Int64(5) AS n -----------EmptyRelation ---Projection: b.x AS count, b.y AS n -----SubqueryAlias: b -------Projection: Int64(1) AS x, MAX(Int64(10)) AS y ---------Aggregate: groupBy=[[]], aggr=[[MAX(Int64(10))]] -----------EmptyRelation +01)Union +02)--Projection: COUNT(*) AS count, a.n +03)----Aggregate: groupBy=[[a.n]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------SubqueryAlias: a +05)--------Projection: Int64(5) AS n +06)----------EmptyRelation +07)--Projection: b.x AS count, b.y AS n +08)----SubqueryAlias: b +09)------Projection: Int64(1) AS x, MAX(Int64(10)) AS y +10)--------Aggregate: groupBy=[[]], aggr=[[MAX(Int64(10))]] +11)----------EmptyRelation physical_plan -UnionExec ---ProjectionExec: expr=[COUNT(*)@1 as count, n@0 as n] -----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[COUNT(*)], ordering_mode=Sorted -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=1 -----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)], ordering_mode=Sorted -------------ProjectionExec: expr=[5 as n] ---------------PlaceholderRowExec ---ProjectionExec: expr=[1 as count, MAX(Int64(10))@0 as n] -----AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] -------PlaceholderRowExec +01)UnionExec +02)--ProjectionExec: expr=[COUNT(*)@1 as count, n@0 as n] +03)----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[COUNT(*)], ordering_mode=Sorted +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)], ordering_mode=Sorted +07)------------ProjectionExec: expr=[5 as n] +08)--------------PlaceholderRowExec +09)--ProjectionExec: expr=[1 as count, MAX(Int64(10))@0 as n] +10)----AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] +11)------PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index 6412c3ca859e..49b2bd9aa0b5 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -30,17 +30,17 @@ query TT explain update t1 set a=1, b=2, c=3.0, d=NULL; ---- logical_plan -Dml: op=[Update] table=[t1] ---Projection: CAST(Int64(1) AS Int32) AS a, CAST(Int64(2) AS Utf8) AS b, Float64(3) AS c, CAST(NULL AS Int32) AS d -----TableScan: t1 +01)Dml: op=[Update] table=[t1] +02)--Projection: CAST(Int64(1) AS Int32) AS a, CAST(Int64(2) AS Utf8) AS b, Float64(3) AS c, CAST(NULL AS Int32) AS d +03)----TableScan: t1 query TT explain update t1 set a=c+1, b=a, c=c+1.0, d=b; ---- logical_plan -Dml: op=[Update] table=[t1] ---Projection: CAST(t1.c + CAST(Int64(1) AS Float64) AS Int32) AS a, CAST(t1.a AS Utf8) AS b, t1.c + Float64(1) AS c, CAST(t1.b AS Int32) AS d -----TableScan: t1 +01)Dml: op=[Update] table=[t1] +02)--Projection: CAST(t1.c + CAST(Int64(1) AS Float64) AS Int32) AS a, CAST(t1.a AS Utf8) AS b, t1.c + Float64(1) AS c, CAST(t1.b AS Int32) AS d +03)----TableScan: t1 statement ok create table t2(a int, b varchar, c double, d int); @@ -50,26 +50,26 @@ query TT explain update t1 set b = (select max(b) from t2 where t1.a = t2.a) ---- logical_plan -Dml: op=[Update] table=[t1] ---Projection: t1.a AS a, () AS b, t1.c AS c, t1.d AS d -----Subquery: -------Projection: MAX(t2.b) ---------Aggregate: groupBy=[[]], aggr=[[MAX(t2.b)]] -----------Filter: outer_ref(t1.a) = t2.a -------------TableScan: t2 -----TableScan: t1 +01)Dml: op=[Update] table=[t1] +02)--Projection: t1.a AS a, () AS b, t1.c AS c, t1.d AS d +03)----Subquery: +04)------Projection: MAX(t2.b) +05)--------Aggregate: groupBy=[[]], aggr=[[MAX(t2.b)]] +06)----------Filter: outer_ref(t1.a) = t2.a +07)------------TableScan: t2 +08)----TableScan: t1 # set from other table query TT explain update t1 set b = t2.b, c = t2.a, d = 1 from t2 where t1.a = t2.a and t1.b > 'foo' and t2.c > 1.0; ---- logical_plan -Dml: op=[Update] table=[t1] ---Projection: t1.a AS a, t2.b AS b, CAST(t2.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d -----Filter: t1.a = t2.a AND t1.b > Utf8("foo") AND t2.c > Float64(1) -------CrossJoin: ---------TableScan: t1 ---------TableScan: t2 +01)Dml: op=[Update] table=[t1] +02)--Projection: t1.a AS a, t2.b AS b, CAST(t2.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d +03)----Filter: t1.a = t2.a AND t1.b > Utf8("foo") AND t2.c > Float64(1) +04)------CrossJoin: +05)--------TableScan: t1 +06)--------TableScan: t2 statement ok create table t3(a int, b varchar, c double, d int); @@ -83,10 +83,10 @@ query TT explain update t1 as T set b = t2.b, c = t.a, d = 1 from t2 where t.a = t2.a and t.b > 'foo' and t2.c > 1.0; ---- logical_plan -Dml: op=[Update] table=[t1] ---Projection: t.a AS a, t2.b AS b, CAST(t.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d -----Filter: t.a = t2.a AND t.b > Utf8("foo") AND t2.c > Float64(1) -------CrossJoin: ---------SubqueryAlias: t -----------TableScan: t1 ---------TableScan: t2 +01)Dml: op=[Update] table=[t1] +02)--Projection: t.a AS a, t2.b AS b, CAST(t.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d +03)----Filter: t.a = t2.a AND t.b > Utf8("foo") AND t2.c > Float64(1) +04)------CrossJoin: +05)--------SubqueryAlias: t +06)----------TableScan: t1 +07)--------TableScan: t2 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 50132e783adb..bcdee9593178 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -253,39 +253,39 @@ WITH _sample_data AS ( ORDER BY d.b; ---- logical_plan -Sort: d.b ASC NULLS LAST ---Projection: d.b, MAX(d.a) AS max_a -----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a)]] -------SubqueryAlias: d ---------SubqueryAlias: _data2 -----------SubqueryAlias: s -------------SubqueryAlias: _sample_data ---------------Union -----------------Projection: Int64(1) AS a, Utf8("aa") AS b -------------------EmptyRelation -----------------Projection: Int64(3) AS a, Utf8("aa") AS b -------------------EmptyRelation -----------------Projection: Int64(5) AS a, Utf8("bb") AS b -------------------EmptyRelation -----------------Projection: Int64(7) AS a, Utf8("bb") AS b -------------------EmptyRelation +01)Sort: d.b ASC NULLS LAST +02)--Projection: d.b, MAX(d.a) AS max_a +03)----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a)]] +04)------SubqueryAlias: d +05)--------SubqueryAlias: _data2 +06)----------SubqueryAlias: s +07)------------SubqueryAlias: _sample_data +08)--------------Union +09)----------------Projection: Int64(1) AS a, Utf8("aa") AS b +10)------------------EmptyRelation +11)----------------Projection: Int64(3) AS a, Utf8("aa") AS b +12)------------------EmptyRelation +13)----------------Projection: Int64(5) AS a, Utf8("bb") AS b +14)------------------EmptyRelation +15)----------------Projection: Int64(7) AS a, Utf8("bb") AS b +16)------------------EmptyRelation physical_plan -SortPreservingMergeExec: [b@0 ASC NULLS LAST] ---SortExec: expr=[b@0 ASC NULLS LAST] -----ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a] -------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] ---------------UnionExec -----------------ProjectionExec: expr=[1 as a, aa as b] -------------------PlaceholderRowExec -----------------ProjectionExec: expr=[3 as a, aa as b] -------------------PlaceholderRowExec -----------------ProjectionExec: expr=[5 as a, bb as b] -------------------PlaceholderRowExec -----------------ProjectionExec: expr=[7 as a, bb as b] -------------------PlaceholderRowExec +01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] +02)--SortExec: expr=[b@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a] +04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] +08)--------------UnionExec +09)----------------ProjectionExec: expr=[1 as a, aa as b] +10)------------------PlaceholderRowExec +11)----------------ProjectionExec: expr=[3 as a, aa as b] +12)------------------PlaceholderRowExec +13)----------------ProjectionExec: expr=[5 as a, bb as b] +14)------------------PlaceholderRowExec +15)----------------ProjectionExec: expr=[7 as a, bb as b] +16)------------------PlaceholderRowExec # Check actual result: query TI @@ -336,42 +336,42 @@ WITH _sample_data AS ( ORDER BY d.b ---- logical_plan -Sort: d.b ASC NULLS LAST ---Projection: d.b, MAX(d.a) AS max_a, MAX(d.seq) -----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a), MAX(d.seq)]] -------SubqueryAlias: d ---------SubqueryAlias: _data2 -----------Projection: ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS seq, s.a, s.b -------------WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------------SubqueryAlias: s -----------------SubqueryAlias: _sample_data -------------------Union ---------------------Projection: Int64(1) AS a, Utf8("aa") AS b -----------------------EmptyRelation ---------------------Projection: Int64(3) AS a, Utf8("aa") AS b -----------------------EmptyRelation ---------------------Projection: Int64(5) AS a, Utf8("bb") AS b -----------------------EmptyRelation ---------------------Projection: Int64(7) AS a, Utf8("bb") AS b -----------------------EmptyRelation +01)Sort: d.b ASC NULLS LAST +02)--Projection: d.b, MAX(d.a) AS max_a, MAX(d.seq) +03)----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a), MAX(d.seq)]] +04)------SubqueryAlias: d +05)--------SubqueryAlias: _data2 +06)----------Projection: ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS seq, s.a, s.b +07)------------WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +08)--------------SubqueryAlias: s +09)----------------SubqueryAlias: _sample_data +10)------------------Union +11)--------------------Projection: Int64(1) AS a, Utf8("aa") AS b +12)----------------------EmptyRelation +13)--------------------Projection: Int64(3) AS a, Utf8("aa") AS b +14)----------------------EmptyRelation +15)--------------------Projection: Int64(5) AS a, Utf8("bb") AS b +16)----------------------EmptyRelation +17)--------------------Projection: Int64(7) AS a, Utf8("bb") AS b +18)----------------------EmptyRelation physical_plan -SortPreservingMergeExec: [b@0 ASC NULLS LAST] ---ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] -----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted -------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 -----------------UnionExec -------------------ProjectionExec: expr=[1 as a, aa as b] ---------------------PlaceholderRowExec -------------------ProjectionExec: expr=[3 as a, aa as b] ---------------------PlaceholderRowExec -------------------ProjectionExec: expr=[5 as a, bb as b] ---------------------PlaceholderRowExec -------------------ProjectionExec: expr=[7 as a, bb as b] ---------------------PlaceholderRowExec +01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] +03)----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted +04)------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] +05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 +09)----------------UnionExec +10)------------------ProjectionExec: expr=[1 as a, aa as b] +11)--------------------PlaceholderRowExec +12)------------------ProjectionExec: expr=[3 as a, aa as b] +13)--------------------PlaceholderRowExec +14)------------------ProjectionExec: expr=[5 as a, bb as b] +15)--------------------PlaceholderRowExec +16)------------------ProjectionExec: expr=[7 as a, bb as b] +17)--------------------PlaceholderRowExec # check actual result @@ -1202,18 +1202,18 @@ EXPLAIN SELECT FROM aggregate_test_100 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 ---WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: aggregate_test_100 projection=[c8, c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 +02)--WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: aggregate_test_100 projection=[c8, c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum2] ---BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum2] +02)--BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true # over_order_by_sort_keys_sorting_prefix_compacting @@ -1223,18 +1223,18 @@ query TT EXPLAIN SELECT c2, MAX(c9) OVER (ORDER BY c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100 ---- logical_plan -Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: aggregate_test_100 projection=[c2, c9] +01)Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan -ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true +01)ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +03)----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true # FIXME: for now we are not detecting prefix of sorting keys in order to re-arrange with global and save one SortExec @@ -1245,21 +1245,21 @@ query TT EXPLAIN SELECT c2, MAX(c9) OVER (ORDER BY c9, c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100 ORDER BY c2 ---- logical_plan -Sort: aggregate_test_100.c2 ASC NULLS LAST ---Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c2, c9] +01)Sort: aggregate_test_100.c2 ASC NULLS LAST +02)--Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan -SortExec: expr=[c2@0 ASC NULLS LAST] ---ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true +01)SortExec: expr=[c2@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +03)----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +04)------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST] +06)----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true # test_window_partition_by_order_by statement ok @@ -1272,24 +1272,24 @@ EXPLAIN SELECT FROM aggregate_test_100 ---- logical_plan -Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING ---WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -----Projection: aggregate_test_100.c1, aggregate_test_100.c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING -------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c1, c2, c4] +01)Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +02)--WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +03)----Projection: aggregate_test_100.c1, aggregate_test_100.c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +04)------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c2, c4] physical_plan -ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] ---BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=4096 ---------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -----------------CoalesceBatchesExec: target_batch_size=4096 -------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true +01)ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +02)--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +03)----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=4096 +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +09)----------------CoalesceBatchesExec: target_batch_size=4096 +10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true # test_window_agg_sort_reversed_plan @@ -1303,18 +1303,18 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query III SELECT @@ -1344,18 +1344,18 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c9] +01)Projection: aggregate_test_100.c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, 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(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, 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] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, 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(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, 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] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query IIIIIII SELECT @@ -1387,19 +1387,19 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c9] +01)Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 ASC NULLS LAST] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c9@0 DESC] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 ASC NULLS LAST] +05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c9@0 DESC] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query III @@ -1428,21 +1428,21 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -----------TableScan: aggregate_test_100 projection=[c1, c2, c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +06)----------TableScan: aggregate_test_100 projection=[c1, c2, c9] physical_plan -ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -------------SortExec: expr=[c9@2 DESC,c1@0 DESC] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true +01)ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +06)----------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[c9@2 DESC,c1@0 DESC] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true query IIII SELECT @@ -1504,36 +1504,36 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS l, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS m, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS n, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS o, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS o11 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] -------Projection: null_cases.c1, null_cases.c3, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] -----------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] -------------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] ---------------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------------------TableScan: null_cases projection=[c1, c2, c3] +01)Projection: SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS l, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS m, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS n, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS o, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS o11 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] +04)------Projection: null_cases.c1, null_cases.c3, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +05)--------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +08)--------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] +09)----------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] +10)------------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING]] +11)--------------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +12)----------------------TableScan: null_cases projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@7 as e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as l, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as m, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@15 as n, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as o, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@9 as e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@10 as m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as o11] ---GlobalLimitExec: skip=0, fetch=5 -----WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] -------ProjectionExec: expr=[c1@0 as c1, c3@2 as c3, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@4 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@6 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@7 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@8 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@10 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@11 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@12 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@14 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@15 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST] -------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC] -----------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC] ---------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }] -----------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] -------------------------SortExec: expr=[c3@2 DESC NULLS LAST] ---------------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] -----------------------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST] ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true +01)ProjectionExec: expr=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@7 as e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as l, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as m, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@15 as n, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as o, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@9 as e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@10 as m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as o11] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] +04)------ProjectionExec: expr=[c1@0 as c1, c3@2 as c3, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@4 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@6 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@7 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@8 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@10 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@11 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@12 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@14 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@15 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +05)--------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST] +07)------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)--------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC] +09)----------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +10)------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC] +11)--------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }] +12)----------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] +13)------------------------SortExec: expr=[c3@2 DESC NULLS LAST] +14)--------------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] +15)----------------------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +16)------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST] +17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -1597,18 +1597,18 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c1, c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] ---GlobalLimitExec: skip=0, fetch=5 - ----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +01)ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III @@ -1641,18 +1641,18 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c1, c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +01)ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1685,24 +1685,24 @@ EXPLAIN SELECT c3, LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------Projection: aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c3, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 -------------TableScan: aggregate_test_100 projection=[c2, c3, c4, c9] +01)Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------Projection: aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c3, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 +07)------------TableScan: aggregate_test_100 projection=[c2, c3, c4, c9] physical_plan -ProjectionExec: expr=[c3@1 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] -------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c3@2 as c3, c9@3 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] ---------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] -----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true +01)ProjectionExec: expr=[c3@1 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] +04)------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c3@2 as c3, c9@3 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +07)------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +08)--------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true query III @@ -1732,31 +1732,31 @@ EXPLAIN SELECT count(*) as global_count FROM ORDER BY c1 ) AS a ---- logical_plan -Projection: COUNT(*) AS global_count ---Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----SubqueryAlias: a -------Projection: ---------Sort: aggregate_test_100.c1 ASC NULLS LAST -----------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] -------------Projection: aggregate_test_100.c1 ---------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") -----------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] +01)Projection: COUNT(*) AS global_count +02)--Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----SubqueryAlias: a +04)------Projection: +05)--------Sort: aggregate_test_100.c1 ASC NULLS LAST +06)----------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] +07)------------Projection: aggregate_test_100.c1 +08)--------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") +09)----------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] physical_plan -ProjectionExec: expr=[COUNT(*)@0 as global_count] ---AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 -----------ProjectionExec: expr=[] -------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] ---------------CoalesceBatchesExec: target_batch_size=4096 -----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] ---------------------ProjectionExec: expr=[c1@0 as c1] -----------------------CoalesceBatchesExec: target_batch_size=4096 -------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +01)ProjectionExec: expr=[COUNT(*)@0 as global_count] +02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 +06)----------ProjectionExec: expr=[] +07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +08)--------------CoalesceBatchesExec: target_batch_size=4096 +09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +10)------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] +11)--------------------ProjectionExec: expr=[c1@0 as c1] +12)----------------------CoalesceBatchesExec: target_batch_size=4096 +13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 +14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true query I SELECT count(*) as global_count FROM @@ -1783,26 +1783,26 @@ EXPLAIN SELECT c3, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -----Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------Projection: aggregate_test_100.c3, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------TableScan: aggregate_test_100 projection=[c2, c3, c9] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +03)----Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------Projection: aggregate_test_100.c3, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +06)----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------TableScan: aggregate_test_100 projection=[c2, c3, c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 -----ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC] -----------CoalesceBatchesExec: target_batch_size=4096 -------------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 +03)----ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC] +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +10)------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +11)--------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true @@ -1829,19 +1829,19 @@ query TT EXPLAIN SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 -----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------TableScan: aggregate_test_100 projection=[c1] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +03)----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ---ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------SortExec: expr=[c1@0 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=4096 -----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c1@0 ASC NULLS LAST] +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -1958,19 +1958,19 @@ query TT EXPLAIN SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 -----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------TableScan: aggregate_test_100 projection=[c1] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +03)----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] ---ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------SortExec: expr=[c1@0 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=4096 -----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c1@0 ASC NULLS LAST] +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -1983,23 +1983,23 @@ EXPLAIN SELECT c1, FROM aggregate_test_100 ORDER BY c1 ASC ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c1, c9] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -SortExec: expr=[c1@0 ASC NULLS LAST] ---ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] ---------SortExec: expr=[c9@1 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] ---------------CoalesceBatchesExec: target_batch_size=4096 -----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +01)SortExec: expr=[c1@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +04)------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST] +06)----------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] +08)--------------CoalesceBatchesExec: target_batch_size=4096 +09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true # test_window_agg_with_global_limit statement ok @@ -2009,20 +2009,20 @@ query TT EXPLAIN SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) ---- logical_plan -Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(aggregate_test_100.c13)]] -----Limit: skip=0, fetch=1 -------Sort: aggregate_test_100.c13 ASC NULLS LAST, fetch=1 ---------TableScan: aggregate_test_100 projection=[c13] +01)Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(aggregate_test_100.c13)]] +03)----Limit: skip=0, fetch=1 +04)------Sort: aggregate_test_100.c13 ASC NULLS LAST, fetch=1 +05)--------TableScan: aggregate_test_100 projection=[c13] physical_plan -ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------GlobalLimitExec: skip=0, fetch=1 -------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +01)ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------GlobalLimitExec: skip=0, fetch=1 +07)------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? @@ -2070,26 +2070,26 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 -----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum4 -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING -----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] -------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] -----------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +03)----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum4 +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING +06)----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] +09)----------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c9@0 ASC NULLS LAST] -----ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c9@3 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] -----------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c9@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +05)--------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c9@3 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] +06)----------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] +07)------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +08)--------------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] +09)----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true @@ -2124,30 +2124,30 @@ EXPLAIN SELECT c9, LIMIT 5 ---- logical_plan -Projection: t1.c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum3, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum4 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------Projection: t1.c2, t1.c9, t1.c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING ---------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] -----------Projection: t1.c2, t1.c8, t1.c9, t1.c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING -------------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] -----------------SubqueryAlias: t1 -------------------Sort: aggregate_test_100.c9 ASC NULLS LAST ---------------------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c8, aggregate_test_100.c9, aggregate_test_100.c1 AS c1_alias -----------------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] +01)Projection: t1.c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum3, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum4 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------Projection: t1.c2, t1.c9, t1.c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING +05)--------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] +06)----------Projection: t1.c2, t1.c8, t1.c9, t1.c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING +07)------------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] +09)----------------SubqueryAlias: t1 +10)------------------Sort: aggregate_test_100.c9 ASC NULLS LAST +11)--------------------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c8, aggregate_test_100.c9, aggregate_test_100.c1 AS c1_alias +12)----------------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] physical_plan -ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -------ProjectionExec: expr=[c2@0 as c2, c9@2 as c9, c1_alias@3 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] ---------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] -------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +01)ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +04)------ProjectionExec: expr=[c2@0 as c2, c9@2 as c9, c1_alias@3 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] +05)--------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] +06)----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +08)--------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] +09)----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +10)------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true query IIIII SELECT c9, @@ -2174,24 +2174,24 @@ EXPLAIN SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FO FROM aggregate_test_100 ORDER BY c9 LIMIT 5 ---- logical_plan -Projection: sum1, sum2 ---Limit: skip=0, fetch=5 -----Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 -------Projection: SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING AS sum2, aggregate_test_100.c9 ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING]] -----------Projection: aggregate_test_100.c1, aggregate_test_100.c9, aggregate_test_100.c12, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING -------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ---------------TableScan: aggregate_test_100 projection=[c1, c2, c9, c12] +01)Projection: sum1, sum2 +02)--Limit: skip=0, fetch=5 +03)----Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +04)------Projection: SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING AS sum2, aggregate_test_100.c9 +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING]] +06)----------Projection: aggregate_test_100.c1, aggregate_test_100.c9, aggregate_test_100.c12, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING +07)------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +08)--------------TableScan: aggregate_test_100 projection=[c1, c2, c9, c12] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] -------ProjectionExec: expr=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING@4 as sum2, c9@1 as c9] ---------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: false }], mode=[Sorted] -----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true +01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING@4 as sum2, c9@1 as c9] +05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: false }], mode=[Sorted] +06)----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, @@ -2216,18 +2216,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 ASC NULLS LAST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 ASC NULLS LAST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 ASC NULLS LAST] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 ASC NULLS LAST] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2255,18 +2255,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2294,19 +2294,19 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 DESC NULLS FIRST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 DESC NULLS FIRST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[rn1@1 DESC] -----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[rn1@1 DESC] +03)----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +04)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2337,19 +2337,19 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] -----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +04)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2391,18 +2391,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true # This test shows that ordering equivalence can keep track of complex expressions (not just Column expressions) # during ordering satisfy analysis. In the final plan we should only see single SortExec. @@ -2415,18 +2415,18 @@ EXPLAIN SELECT c5, c9, rn1 FROM (SELECT c5, c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST, fetch=5 -----Sort: CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST -------Projection: aggregate_test_100.c5, aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c5, c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST, fetch=5 +03)----Sort: CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c5, aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c5, c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c5@0 as c5, c9@1 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[CAST(c9@1 AS Int32) + c5@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c5@0 as c5, c9@1 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[CAST(c9@1 AS Int32) + c5@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], has_header=true # Ordering equivalence should be preserved during cast expression query TT @@ -2438,18 +2438,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS Int64) AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS Int64) AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 AS Int64) as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 AS Int64) as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true # The following query has type error. We should test the error could be detected # from either the logical plan (when `skip_failed_rules` is set to `false`) or @@ -2540,27 +2540,27 @@ EXPLAIN SELECT LIMIT 5; ---- logical_plan -Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, sumr1, sumr2, sumr3, minr1, minr2, minr3, maxr1, maxr2, maxr3, cntr1, cntr2, sum4, cnt3 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.inc_col DESC NULLS FIRST, fetch=5 -------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING AS sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING AS sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt3, annotated_data_finite.inc_col ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] -----------Projection: CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, annotated_data_finite.inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING -------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] -----------------Projection: CAST(annotated_data_finite.desc_col AS Int64) AS CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(annotated_data_finite.inc_col AS Int64) AS CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, annotated_data_finite.ts, annotated_data_finite.inc_col, annotated_data_finite.desc_col -------------------TableScan: annotated_data_finite projection=[ts, inc_col, desc_col] +01)Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, sumr1, sumr2, sumr3, minr1, minr2, minr3, maxr1, maxr2, maxr3, cntr1, cntr2, sum4, cnt3 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_finite.inc_col DESC NULLS FIRST, fetch=5 +04)------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING AS sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING AS sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt3, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +06)----------Projection: CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, annotated_data_finite.inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING +07)------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +09)----------------Projection: CAST(annotated_data_finite.desc_col AS Int64) AS CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(annotated_data_finite.inc_col AS Int64) AS CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, annotated_data_finite.ts, annotated_data_finite.inc_col, annotated_data_finite.desc_col +10)------------------TableScan: annotated_data_finite projection=[ts, inc_col, desc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] -------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@14 as sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@15 as sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@22 as cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@23 as cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@2 as sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@3 as sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@6 as minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@11 as cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@12 as cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as cnt3, inc_col@1 as inc_col] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col@0 as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, inc_col@3 as inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] -----------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] +04)------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@14 as sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@15 as sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@22 as cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@23 as cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@2 as sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@3 as sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@6 as minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@11 as cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@12 as cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as cnt3, inc_col@1 as inc_col] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +06)----------ProjectionExec: expr=[CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col@0 as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, inc_col@3 as inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +08)--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] +09)----------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2633,19 +2633,19 @@ EXPLAIN SELECT LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 -----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 -------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]] ---------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]] -----------TableScan: annotated_data_finite projection=[ts, inc_col] +01)Limit: skip=0, fetch=5 +02)--Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 +03)----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 +04)------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]] +05)--------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]] +06)----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[ts@0 DESC] -----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] -------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] ---------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] -----------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 +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[ts@0 DESC] +03)----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] +04)------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] +05)--------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] +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 query IIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2704,23 +2704,23 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS avg2, annotated_data_finite.inc_col ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(annotated_data_finite.inc_col AS Int64) AS CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, annotated_data_finite.ts, annotated_data_finite.inc_col ---------------TableScan: annotated_data_finite projection=[ts, inc_col] +01)Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 +04)------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS avg2, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(annotated_data_finite.inc_col AS Int64) AS CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, annotated_data_finite.ts, annotated_data_finite.inc_col +08)--------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] -------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@12 as count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@7 as count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@13 as avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@8 as avg2, inc_col@3 as inc_col] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] -------------ProjectionExec: expr=[CAST(inc_col@1 AS Float64) as CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col] ---------------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 +01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] +04)------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@12 as count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@7 as count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@13 as avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@8 as avg2, inc_col@3 as inc_col] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] +07)------------ProjectionExec: expr=[CAST(inc_col@1 AS Float64) as CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIRR SELECT @@ -2759,21 +2759,21 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -------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 ---------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]] -----------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]] -------------TableScan: annotated_data_finite projection=[ts, inc_col] +01)Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 +04)------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 +05)--------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]] +06)----------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]] +07)------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -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] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] -------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] ---------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] -----------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] -------------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 +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)--GlobalLimitExec: skip=0, fetch=5 +03)----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] +04)------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] +05)--------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] +06)----------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] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIII SELECT @@ -2805,22 +2805,22 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: sum1, sum2, count1, count2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 -------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col ---------------TableScan: annotated_data_infinite projection=[ts, inc_col] +01)Projection: sum1, sum2, count1, count2 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 +04)------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col +08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2852,22 +2852,22 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: sum1, sum2, count1, count2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 -------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col ---------------TableScan: annotated_data_infinite projection=[ts, inc_col] +01)Projection: sum1, sum2, count1, count2 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 +04)------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col +08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] +06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2951,27 +2951,27 @@ EXPLAIN SELECT a, b, c, LIMIT 5 ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] -------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] ---------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -----------------Projection: CAST(annotated_data_infinite2.c AS Int64) AS CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d -------------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] +07)------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] +08)--------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +09)----------------Projection: CAST(annotated_data_infinite2.c AS Int64) AS CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +10)------------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Linear] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[PartiallySorted([1, 0])] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] -------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Linear] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[PartiallySorted([1, 0])] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] +07)------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] +08)--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +09)----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +10)------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3019,34 +3019,34 @@ EXPLAIN SELECT a, b, c, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: annotated_data_finite2.c ASC NULLS LAST, fetch=5 -----Projection: annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] -----------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -------------------Projection: CAST(annotated_data_finite2.c AS Int64) AS CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, annotated_data_finite2.d ---------------------TableScan: annotated_data_finite2 projection=[a, b, c, d] +01)Limit: skip=0, fetch=5 +02)--Sort: annotated_data_finite2.c ASC NULLS LAST, fetch=5 +03)----Projection: annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +07)------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] +09)----------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +10)------------------Projection: CAST(annotated_data_finite2.c AS Int64) AS CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, annotated_data_finite2.d +11)--------------------TableScan: annotated_data_finite2 projection=[a, b, c, d] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST] -----ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Sorted] ---------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -----------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST] -------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] -------------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] ---------------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST] +03)----ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Sorted] +05)--------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] +08)--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +09)----------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST] +10)------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +11)--------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +12)----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] +13)------------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] +14)--------------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +15)----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3098,20 +3098,20 @@ EXPLAIN SELECT * FROM (SELECT *, ROW_NUMBER() OVER(ORDER BY a ASC) as rn1 ORDER BY rn1 ASC ---- logical_plan -Sort: rn1 ASC NULLS LAST ---Filter: rn1 < UInt64(50) -----Limit: skip=0, fetch=5 -------Sort: rn1 ASC NULLS LAST, fetch=5 ---------Projection: annotated_data_infinite2.a0, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 -----------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] +01)Sort: rn1 ASC NULLS LAST +02)--Filter: rn1 < UInt64(50) +03)----Limit: skip=0, fetch=5 +04)------Sort: rn1 ASC NULLS LAST, fetch=5 +05)--------Projection: annotated_data_infinite2.a0, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +06)----------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=4096 ---FilterExec: rn1@5 < 50 -----GlobalLimitExec: skip=0, fetch=5 -------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +01)CoalesceBatchesExec: target_batch_size=4096 +02)--FilterExec: rn1@5 < 50 +03)----GlobalLimitExec: skip=0, fetch=5 +04)------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] +05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] # this is a negative test for asserting that window functions (other than ROW_NUMBER) # are not added to ordering equivalence @@ -3125,19 +3125,19 @@ EXPLAIN SELECT c9, sum1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: sum1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: sum1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] -----ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] +03)----ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true # Query below should work when its input is unbounded # because ordering of ROW_NUMBER, RANK result is added to the ordering equivalence @@ -3210,23 +3210,23 @@ SUM(a) OVER(partition by b, a order by c) as sum2, FROM annotated_data_infinite2; ---- logical_plan -Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 ---WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d ---------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +01)Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 +02)--WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +08)--------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] ---BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] -----ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] ---------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] +02)--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] +03)----ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +08)--------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] statement ok set datafusion.execution.target_partitions = 2; @@ -3241,32 +3241,32 @@ EXPLAIN SELECT SUM(a) OVER(partition by a, b order by c) as sum1, FROM annotated_data_infinite2; ---- logical_plan -Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 ---WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d ---------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +01)Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 +02)--WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +08)--------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] ---BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] -----CoalesceBatchesExec: target_batch_size=4096 -------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST,a@1 ASC NULLS LAST ---------ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------CoalesceBatchesExec: target_batch_size=4096 ---------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] -------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] +02)--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST,a@1 ASC NULLS LAST +05)--------ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------CoalesceBatchesExec: target_batch_size=4096 +08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST +09)----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] +10)------------------CoalesceBatchesExec: target_batch_size=4096 +11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST +12)----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +13)------------------------CoalesceBatchesExec: target_batch_size=4096 +14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST +15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] # reset the partition number 1 again statement ok @@ -3311,23 +3311,23 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 -------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING -----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------------TableScan: aggregate_test_100 projection=[c3, c11, c12] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +03)----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 +04)------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +06)----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------TableScan: aggregate_test_100 projection=[c3, c11, c12] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] -----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] -------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c12@1 ASC NULLS LAST] -----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] ---------------SortExec: expr=[c11@1 ASC NULLS LAST] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] +04)------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c12@1 ASC NULLS LAST] +06)----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +07)------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +08)--------------SortExec: expr=[c11@1 ASC NULLS LAST] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. @@ -3357,20 +3357,20 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: min1, max1 ---Limit: skip=0, fetch=5 -----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1, aggregate_test_100.c3 ---------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c3, c12] +01)Projection: min1, max1 +02)--Limit: skip=0, fetch=5 +03)----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +04)------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1, aggregate_test_100.c3 +05)--------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c3, c12] physical_plan -ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] -------ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] ---------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c12@1 ASC NULLS LAST] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true +01)ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] +05)--------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c12@1 ASC NULLS LAST] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true # window2 spec is not defined statement error DataFusion error: Error during planning: The window window2 is not defined! @@ -3416,14 +3416,14 @@ FROM multiple_ordered_table where b=0 ---- logical_plan -WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---Filter: multiple_ordered_table.b = Int32(0) -----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +01)WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +02)--Filter: multiple_ordered_table.b = Int32(0) +03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan -BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---CoalesceBatchesExec: target_batch_size=4096 -----FilterExec: b@2 = 0 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +01)BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +02)--CoalesceBatchesExec: target_batch_size=4096 +03)----FilterExec: b@2 = 0 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3434,15 +3434,15 @@ FROM multiple_ordered_table where b=0 ---- logical_plan -WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---Filter: multiple_ordered_table.b = Int32(0) -----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +01)WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +02)--Filter: multiple_ordered_table.b = Int32(0) +03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan -BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---SortExec: expr=[d@4 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 -------FilterExec: b@2 = 0 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +01)BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +02)--SortExec: expr=[d@4 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------FilterExec: b@2 = 0 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true # Create an unbounded source where there is multiple orderings. @@ -3468,17 +3468,17 @@ EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, FROM multiple_ordered_table ---- logical_plan -Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 ---WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: multiple_ordered_table projection=[a, b, c, d] +01)Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 +02)--WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] ---BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] +02)--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c @@ -3488,42 +3488,42 @@ FROM( WHERE d=0) ---- logical_plan -Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c ---WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Filter: multiple_ordered_table.d = Int32(0) -------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] +01)Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c +02)--WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Filter: multiple_ordered_table.d = Int32(0) +04)------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] physical_plan -ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] ---BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CoalesceBatchesExec: target_batch_size=4096 -------FilterExec: d@1 = 0 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] +02)--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------FilterExec: d@1 = 0 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) FROM multiple_ordered_table; ---- logical_plan -Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----TableScan: multiple_ordered_table projection=[a, c, d] +01)Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) FROM multiple_ordered_table; ---- logical_plan -Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----TableScan: multiple_ordered_table projection=[a, b, c, d] +01)Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query I SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3557,16 +3557,16 @@ EXPLAIN SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: multiple_ordered_table.c ASC NULLS LAST, fetch=5 -----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 -------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]] ---------TableScan: multiple_ordered_table projection=[c] +01)Limit: skip=0, fetch=5 +02)--Sort: multiple_ordered_table.c ASC NULLS LAST, fetch=5 +03)----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 +04)------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]] +05)--------TableScan: multiple_ordered_table projection=[c] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---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] -----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 }] -------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 +01)GlobalLimitExec: skip=0, fetch=5 +02)--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] +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 SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 @@ -3609,18 +3609,18 @@ WINDOW sliding_window AS ( ORDER BY c ---- logical_plan -Sort: multiple_ordered_table_inf.c ASC NULLS LAST ---Projection: multiple_ordered_table_inf.a0, multiple_ordered_table_inf.a, multiple_ordered_table_inf.b, multiple_ordered_table_inf.c, multiple_ordered_table_inf.d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW AS avg_d -----WindowAggr: windowExpr=[[AVG(CAST(multiple_ordered_table_inf.d AS Float64)) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW]] -------TableScan: multiple_ordered_table_inf projection=[a0, a, b, c, d] +01)Sort: multiple_ordered_table_inf.c ASC NULLS LAST +02)--Projection: multiple_ordered_table_inf.a0, multiple_ordered_table_inf.a, multiple_ordered_table_inf.b, multiple_ordered_table_inf.c, multiple_ordered_table_inf.d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW AS avg_d +03)----WindowAggr: windowExpr=[[AVG(CAST(multiple_ordered_table_inf.d AS Float64)) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW]] +04)------TableScan: multiple_ordered_table_inf projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [c@3 ASC NULLS LAST] ---ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] -----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] -------CoalesceBatchesExec: target_batch_size=4096 ---------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] +01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] +02)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] +03)----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] +04)------CoalesceBatchesExec: target_batch_size=4096 +05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] # CTAS with NTILE function statement ok @@ -3945,14 +3945,14 @@ query TT EXPLAIN SELECT *, SUM(amount) OVER (ORDER BY sn) as sum1 FROM table_with_pk; ---- logical_plan -Projection: table_with_pk.sn, table_with_pk.ts, table_with_pk.currency, table_with_pk.amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 ---WindowAggr: windowExpr=[[SUM(CAST(table_with_pk.amount AS Float64)) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----TableScan: table_with_pk projection=[sn, ts, currency, amount] +01)Projection: table_with_pk.sn, table_with_pk.ts, table_with_pk.currency, table_with_pk.amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 +02)--WindowAggr: windowExpr=[[SUM(CAST(table_with_pk.amount AS Float64)) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----TableScan: table_with_pk projection=[sn, ts, currency, amount] physical_plan -ProjectionExec: expr=[sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1] ---BoundedWindowAggExec: wdw=[SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, 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] -----SortExec: expr=[sn@0 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1] +02)--BoundedWindowAggExec: wdw=[SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, 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] +03)----SortExec: expr=[sn@0 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] # test ROW_NUMBER window function returns correct data_type query T @@ -4060,20 +4060,20 @@ explain SELECT c3, limit 5 ---- logical_plan -Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, sum1 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------Projection: aggregate_test_100.c3, aggregate_test_100.c4, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c3, c4, c9] +01)Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, sum1 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------Projection: aggregate_test_100.c3, aggregate_test_100.c4, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c3, c4, c9] physical_plan -ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, sum1@3 as sum1] ---GlobalLimitExec: skip=0, fetch=5 -----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] -------ProjectionExec: expr=[c3@0 as c3, c4@1 as c4, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1] ---------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------SortExec: expr=[c3@0 + c4@1 DESC] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true +01)ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, sum1@3 as sum1] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] +04)------ProjectionExec: expr=[c3@0 as c3, c4@1 as c4, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1] +05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[c3@0 + c4@1 DESC] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true query III SELECT c3, @@ -4104,19 +4104,19 @@ query TT EXPLAIN select count(*) over (partition by a order by a) from (select * from a where a = 1); ---- logical_plan -Projection: COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Filter: a.a = Int64(1) -------TableScan: a projection=[a] +01)Projection: COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Filter: a.a = Int64(1) +04)------TableScan: a projection=[a] physical_plan -ProjectionExec: expr=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CoalesceBatchesExec: target_batch_size=4096 -------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------CoalesceBatchesExec: target_batch_size=4096 -------------FilterExec: a@0 = 1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------FilterExec: a@0 = 1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4127,19 +4127,19 @@ query TT EXPLAIN select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); ---- logical_plan -Projection: ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING ---WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----Filter: a.a = Int64(1) -------TableScan: a projection=[a] +01)Projection: ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +02)--WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----Filter: a.a = Int64(1) +04)------TableScan: a projection=[a] physical_plan -ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] ---BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -----CoalesceBatchesExec: target_batch_size=4096 -------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------CoalesceBatchesExec: target_batch_size=4096 -------------FilterExec: a@0 = 1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +02)--BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------FilterExec: a@0 = 1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT From 36f6e0facc8cdbded3e595cbcb441190d055e56d Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Mon, 15 Apr 2024 13:27:03 +0300 Subject: [PATCH 33/39] Use PhysicalExtensionCodec consistently (#10075) * Use PhysicalExtensionCodec consistently * Use PhysicalExtensionCodec consisdently also when serializing * Add a test for window aggregation with UDF codec * Commit binary incompatible changes --- .../proto/src/physical_plan/from_proto.rs | 71 ++- datafusion/proto/src/physical_plan/mod.rs | 93 ++-- .../proto/src/physical_plan/to_proto.rs | 504 ++++++++---------- .../tests/cases/roundtrip_physical_plan.rs | 128 +++-- 4 files changed, 402 insertions(+), 394 deletions(-) diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index aaca4dc48236..81e4c92ffc68 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -21,13 +21,11 @@ use std::collections::HashMap; use std::convert::{TryFrom, TryInto}; use std::sync::Arc; -use crate::common::proto_error; -use crate::convert_required; -use crate::logical_plan::{self, csv_writer_options_from_proto}; -use crate::protobuf::physical_expr_node::ExprType; -use crate::protobuf::{self, copy_to_node}; - use arrow::compute::SortOptions; +use chrono::{TimeZone, Utc}; +use object_store::path::Path; +use object_store::ObjectMeta; + use datafusion::arrow::datatypes::Schema; use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::json::JsonSink; @@ -57,13 +55,15 @@ use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{not_impl_err, DataFusionError, JoinSide, Result, ScalarValue}; - -use chrono::{TimeZone, Utc}; use datafusion_expr::ScalarFunctionDefinition; -use object_store::path::Path; -use object_store::ObjectMeta; -use super::{DefaultPhysicalExtensionCodec, PhysicalExtensionCodec}; +use crate::common::proto_error; +use crate::convert_required; +use crate::logical_plan::{self, csv_writer_options_from_proto}; +use crate::protobuf::physical_expr_node::ExprType; +use crate::protobuf::{self, copy_to_node}; + +use super::PhysicalExtensionCodec; impl From<&protobuf::PhysicalColumn> for Column { fn from(c: &protobuf::PhysicalColumn) -> Column { @@ -76,9 +76,10 @@ impl From<&protobuf::PhysicalColumn> for Column { /// # Arguments /// /// * `proto` - Input proto with physical sort expression node -/// * `registry` - A registry knows how to build logical expressions out of user-defined function' names +/// * `registry` - A registry knows how to build logical expressions out of user-defined function names /// * `input_schema` - The Arrow schema for the input, used for determining expression data types /// when performing type coercion. +/// * `codec` - An extension codec used to decode custom UDFs. pub fn parse_physical_sort_expr( proto: &protobuf::PhysicalSortExprNode, registry: &dyn FunctionRegistry, @@ -102,9 +103,10 @@ pub fn parse_physical_sort_expr( /// # Arguments /// /// * `proto` - Input proto with vector of physical sort expression node -/// * `registry` - A registry knows how to build logical expressions out of user-defined function' names +/// * `registry` - A registry knows how to build logical expressions out of user-defined function names /// * `input_schema` - The Arrow schema for the input, used for determining expression data types /// when performing type coercion. +/// * `codec` - An extension codec used to decode custom UDFs. pub fn parse_physical_sort_exprs( proto: &[protobuf::PhysicalSortExprNode], registry: &dyn FunctionRegistry, @@ -123,25 +125,26 @@ pub fn parse_physical_sort_exprs( /// /// # Arguments /// -/// * `proto` - Input proto with physical window exprression node. +/// * `proto` - Input proto with physical window expression node. /// * `name` - Name of the window expression. -/// * `registry` - A registry knows how to build logical expressions out of user-defined function' names +/// * `registry` - A registry knows how to build logical expressions out of user-defined function names /// * `input_schema` - The Arrow schema for the input, used for determining expression data types /// when performing type coercion. +/// * `codec` - An extension codec used to decode custom UDFs. pub fn parse_physical_window_expr( proto: &protobuf::PhysicalWindowExprNode, registry: &dyn FunctionRegistry, input_schema: &Schema, + codec: &dyn PhysicalExtensionCodec, ) -> Result> { - let codec = DefaultPhysicalExtensionCodec {}; let window_node_expr = - parse_physical_exprs(&proto.args, registry, input_schema, &codec)?; + parse_physical_exprs(&proto.args, registry, input_schema, codec)?; let partition_by = - parse_physical_exprs(&proto.partition_by, registry, input_schema, &codec)?; + parse_physical_exprs(&proto.partition_by, registry, input_schema, codec)?; let order_by = - parse_physical_sort_exprs(&proto.order_by, registry, input_schema, &codec)?; + parse_physical_sort_exprs(&proto.order_by, registry, input_schema, codec)?; let window_frame = proto .window_frame @@ -187,9 +190,10 @@ where /// # Arguments /// /// * `proto` - Input proto with physical expression node -/// * `registry` - A registry knows how to build logical expressions out of user-defined function' names +/// * `registry` - A registry knows how to build logical expressions out of user-defined function names /// * `input_schema` - The Arrow schema for the input, used for determining expression data types /// when performing type coercion. +/// * `codec` - An extension codec used to decode custom UDFs. pub fn parse_physical_expr( proto: &protobuf::PhysicalExprNode, registry: &dyn FunctionRegistry, @@ -213,6 +217,7 @@ pub fn parse_physical_expr( registry, "left", input_schema, + codec, )?, logical_plan::from_proto::from_proto_binary_op(&binary_expr.op)?, parse_required_physical_expr( @@ -220,6 +225,7 @@ pub fn parse_physical_expr( registry, "right", input_schema, + codec, )?, )), ExprType::AggregateExpr(_) => { @@ -241,6 +247,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?)) } ExprType::IsNotNullExpr(e) => { @@ -249,6 +256,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?)) } ExprType::NotExpr(e) => Arc::new(NotExpr::new(parse_required_physical_expr( @@ -256,6 +264,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?)), ExprType::Negative(e) => { Arc::new(NegativeExpr::new(parse_required_physical_expr( @@ -263,6 +272,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?)) } ExprType::InList(e) => in_list( @@ -271,6 +281,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?, parse_physical_exprs(&e.list, registry, input_schema, codec)?, &e.negated, @@ -290,12 +301,14 @@ pub fn parse_physical_expr( registry, "when_expr", input_schema, + codec, )?, parse_required_physical_expr( e.then_expr.as_ref(), registry, "then_expr", input_schema, + codec, )?, )) }) @@ -311,6 +324,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?, convert_required!(e.arrow_type)?, None, @@ -321,6 +335,7 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?, convert_required!(e.arrow_type)?, )), @@ -371,12 +386,14 @@ pub fn parse_physical_expr( registry, "expr", input_schema, + codec, )?, parse_required_physical_expr( like_expr.pattern.as_deref(), registry, "pattern", input_schema, + codec, )?, )), }; @@ -389,9 +406,9 @@ fn parse_required_physical_expr( registry: &dyn FunctionRegistry, field: &str, input_schema: &Schema, + codec: &dyn PhysicalExtensionCodec, ) -> Result> { - let codec = DefaultPhysicalExtensionCodec {}; - expr.map(|e| parse_physical_expr(e, registry, input_schema, &codec)) + expr.map(|e| parse_physical_expr(e, registry, input_schema, codec)) .transpose()? .ok_or_else(|| { DataFusionError::Internal(format!("Missing required field {field:?}")) @@ -433,15 +450,15 @@ pub fn parse_protobuf_hash_partitioning( partitioning: Option<&protobuf::PhysicalHashRepartition>, registry: &dyn FunctionRegistry, input_schema: &Schema, + codec: &dyn PhysicalExtensionCodec, ) -> Result> { match partitioning { Some(hash_part) => { - let codec = DefaultPhysicalExtensionCodec {}; let expr = parse_physical_exprs( &hash_part.hash_expr, registry, input_schema, - &codec, + codec, )?; Ok(Some(Partitioning::Hash( @@ -456,6 +473,7 @@ pub fn parse_protobuf_hash_partitioning( pub fn parse_protobuf_file_scan_config( proto: &protobuf::FileScanExecConf, registry: &dyn FunctionRegistry, + codec: &dyn PhysicalExtensionCodec, ) -> Result { let schema: Arc = Arc::new(convert_required!(proto.schema)?); let projection = proto @@ -489,7 +507,7 @@ pub fn parse_protobuf_file_scan_config( .collect::>>()?; // Remove partition columns from the schema after recreating table_partition_cols - // because the partition columns are not in the file. They are present to allow the + // because the partition columns are not in the file. They are present to allow // the partition column types to be reconstructed after serde. let file_schema = Arc::new(Schema::new( schema @@ -502,12 +520,11 @@ pub fn parse_protobuf_file_scan_config( let mut output_ordering = vec![]; for node_collection in &proto.output_ordering { - let codec = DefaultPhysicalExtensionCodec {}; let sort_expr = parse_physical_sort_exprs( &node_collection.physical_sort_expr_nodes, registry, &schema, - &codec, + codec, )?; output_ordering.push(sort_expr); } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 4d95c847bf99..a481e7090fb3 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -19,22 +19,8 @@ use std::convert::TryInto; use std::fmt::Debug; use std::sync::Arc; -use self::from_proto::parse_physical_window_expr; -use self::to_proto::serialize_physical_expr; - -use crate::common::{byte_to_string, proto_error, str_to_byte}; -use crate::convert_required; -use crate::physical_plan::from_proto::{ - parse_physical_expr, parse_physical_sort_expr, parse_physical_sort_exprs, - parse_protobuf_file_scan_config, -}; -use crate::protobuf::physical_aggregate_expr_node::AggregateFunction; -use crate::protobuf::physical_expr_node::ExprType; -use crate::protobuf::physical_plan_node::PhysicalPlanType; -use crate::protobuf::repartition_exec_node::PartitionMethod; -use crate::protobuf::{ - self, window_agg_exec_node, PhysicalPlanNode, PhysicalSortExprNodeCollection, -}; +use prost::bytes::BufMut; +use prost::Message; use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; @@ -79,13 +65,28 @@ use datafusion::physical_plan::{ use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ScalarUDF; -use prost::bytes::BufMut; -use prost::Message; +use crate::common::{byte_to_string, proto_error, str_to_byte}; +use crate::convert_required; +use crate::physical_plan::from_proto::{ + parse_physical_expr, parse_physical_sort_expr, parse_physical_sort_exprs, + parse_physical_window_expr, parse_protobuf_file_scan_config, +}; +use crate::physical_plan::to_proto::{ + serialize_file_scan_config, serialize_maybe_filter, serialize_physical_aggr_expr, + serialize_physical_window_expr, +}; +use crate::protobuf::physical_aggregate_expr_node::AggregateFunction; +use crate::protobuf::physical_expr_node::ExprType; +use crate::protobuf::physical_plan_node::PhysicalPlanType; +use crate::protobuf::repartition_exec_node::PartitionMethod; +use crate::protobuf::{self, window_agg_exec_node}; + +use self::to_proto::serialize_physical_expr; pub mod from_proto; pub mod to_proto; -impl AsExecutionPlan for PhysicalPlanNode { +impl AsExecutionPlan for protobuf::PhysicalPlanNode { fn try_decode(buf: &[u8]) -> Result where Self: Sized, @@ -191,6 +192,7 @@ impl AsExecutionPlan for PhysicalPlanNode { parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), registry, + extension_codec, )?, scan.has_header, str_to_byte(&scan.delimiter, "delimiter")?, @@ -210,6 +212,7 @@ impl AsExecutionPlan for PhysicalPlanNode { let base_config = parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), registry, + extension_codec, )?; let predicate = scan .predicate @@ -234,6 +237,7 @@ impl AsExecutionPlan for PhysicalPlanNode { Ok(Arc::new(AvroExec::new(parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), registry, + extension_codec, )?))) } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { @@ -338,6 +342,7 @@ impl AsExecutionPlan for PhysicalPlanNode { window_expr, registry, input_schema.as_ref(), + extension_codec, ) }) .collect::, _>>()?; @@ -1449,14 +1454,17 @@ impl AsExecutionPlan for PhysicalPlanNode { let filter = exec .filter_expr() .iter() - .map(|expr| expr.to_owned().try_into()) + .map(|expr| serialize_maybe_filter(expr.to_owned(), extension_codec)) .collect::>>()?; let agg = exec .aggr_expr() .iter() - .map(|expr| expr.to_owned().try_into()) + .map(|expr| { + serialize_physical_aggr_expr(expr.to_owned(), extension_codec) + }) .collect::>>()?; + let agg_names = exec .aggr_expr() .iter() @@ -1556,7 +1564,10 @@ impl AsExecutionPlan for PhysicalPlanNode { return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CsvScan( protobuf::CsvScanExecNode { - base_conf: Some(exec.base_config().try_into()?), + base_conf: Some(serialize_file_scan_config( + exec.base_config(), + extension_codec, + )?), has_header: exec.has_header(), delimiter: byte_to_string(exec.delimiter(), "delimiter")?, quote: byte_to_string(exec.quote(), "quote")?, @@ -1581,7 +1592,10 @@ impl AsExecutionPlan for PhysicalPlanNode { return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( protobuf::ParquetScanExecNode { - base_conf: Some(exec.base_config().try_into()?), + base_conf: Some(serialize_file_scan_config( + exec.base_config(), + extension_codec, + )?), predicate, }, )), @@ -1592,7 +1606,10 @@ impl AsExecutionPlan for PhysicalPlanNode { return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::AvroScan( protobuf::AvroScanExecNode { - base_conf: Some(exec.base_config().try_into()?), + base_conf: Some(serialize_file_scan_config( + exec.base_config(), + extension_codec, + )?), }, )), }); @@ -1688,7 +1705,7 @@ impl AsExecutionPlan for PhysicalPlanNode { } if let Some(union) = plan.downcast_ref::() { - let mut inputs: Vec = vec![]; + let mut inputs: Vec = vec![]; for input in union.inputs() { inputs.push(protobuf::PhysicalPlanNode::try_from_physical_plan( input.to_owned(), @@ -1703,7 +1720,7 @@ impl AsExecutionPlan for PhysicalPlanNode { } if let Some(interleave) = plan.downcast_ref::() { - let mut inputs: Vec = vec![]; + let mut inputs: Vec = vec![]; for input in interleave.inputs() { inputs.push(protobuf::PhysicalPlanNode::try_from_physical_plan( input.to_owned(), @@ -1809,11 +1826,11 @@ impl AsExecutionPlan for PhysicalPlanNode { extension_codec, )?; - let window_expr = - exec.window_expr() - .iter() - .map(|e| e.clone().try_into()) - .collect::>>()?; + let window_expr = exec + .window_expr() + .iter() + .map(|e| serialize_physical_window_expr(e.clone(), extension_codec)) + .collect::>>()?; let partition_keys = exec .partition_keys @@ -1839,11 +1856,11 @@ impl AsExecutionPlan for PhysicalPlanNode { extension_codec, )?; - let window_expr = - exec.window_expr() - .iter() - .map(|e| e.clone().try_into()) - .collect::>>()?; + let window_expr = exec + .window_expr() + .iter() + .map(|e| serialize_physical_window_expr(e.clone(), extension_codec)) + .collect::>>()?; let partition_keys = exec .partition_keys @@ -1901,7 +1918,7 @@ impl AsExecutionPlan for PhysicalPlanNode { Ok(sort_expr) }) .collect::>>()?; - Some(PhysicalSortExprNodeCollection { + Some(protobuf::PhysicalSortExprNodeCollection { physical_sort_expr_nodes: expr, }) } @@ -2044,7 +2061,7 @@ impl PhysicalExtensionCodec for DefaultPhysicalExtensionCodec { } fn into_physical_plan( - node: &Option>, + node: &Option>, registry: &dyn FunctionRegistry, runtime: &RuntimeEnv, extension_codec: &dyn PhysicalExtensionCodec, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index e1574f48fb8e..b4c23e4d0c3c 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -22,18 +22,8 @@ use std::{ sync::Arc, }; -use crate::protobuf::{ - self, copy_to_node, physical_aggregate_expr_node, physical_window_expr_node, - scalar_value::Value, ArrowOptions, AvroOptions, PhysicalSortExprNode, - PhysicalSortExprNodeCollection, ScalarValue, -}; - #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; - -use datafusion_expr::ScalarFunctionDefinition; - -use crate::logical_plan::csv_writer_options_to_proto; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; @@ -71,195 +61,187 @@ use datafusion_common::{ stats::Precision, DataFusionError, JoinSide, Result, }; +use datafusion_expr::ScalarFunctionDefinition; -use super::{DefaultPhysicalExtensionCodec, PhysicalExtensionCodec}; - -impl TryFrom> for protobuf::PhysicalExprNode { - type Error = DataFusionError; - - fn try_from(a: Arc) -> Result { - let codec = DefaultPhysicalExtensionCodec {}; - let expressions = serialize_physical_exprs(a.expressions(), &codec)?; - - let ordering_req = a.order_bys().unwrap_or(&[]).to_vec(); - let ordering_req = serialize_physical_sort_exprs(ordering_req, &codec)?; - - if let Some(a) = a.as_any().downcast_ref::() { - let name = a.fun().name().to_string(); - return Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( - protobuf::PhysicalAggregateExprNode { - aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), - expr: expressions, - ordering_req, - distinct: false, - }, - )), - }); - } +use crate::logical_plan::csv_writer_options_to_proto; +use crate::protobuf::{ + self, copy_to_node, physical_aggregate_expr_node, physical_window_expr_node, + scalar_value::Value, ArrowOptions, AvroOptions, PhysicalSortExprNode, + PhysicalSortExprNodeCollection, ScalarValue, +}; - let AggrFn { - inner: aggr_function, - distinct, - } = aggr_expr_to_aggr_fn(a.as_ref())?; +use super::PhysicalExtensionCodec; - Ok(protobuf::PhysicalExprNode { +pub fn serialize_physical_aggr_expr( + aggr_expr: Arc, + codec: &dyn PhysicalExtensionCodec, +) -> Result { + let expressions = serialize_physical_exprs(aggr_expr.expressions(), codec)?; + let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); + let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; + + if let Some(a) = aggr_expr.as_any().downcast_ref::() { + let name = a.fun().name().to_string(); + return Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( protobuf::PhysicalAggregateExprNode { - aggregate_function: Some( - physical_aggregate_expr_node::AggregateFunction::AggrFunction( - aggr_function as i32, - ), - ), + aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), expr: expressions, ordering_req, - distinct, + distinct: false, }, )), - }) + }); } -} - -impl TryFrom> for protobuf::PhysicalWindowExprNode { - type Error = DataFusionError; - fn try_from( - window_expr: Arc, - ) -> std::result::Result { - let expr = window_expr.as_any(); + let AggrFn { + inner: aggr_function, + distinct, + } = aggr_expr_to_aggr_fn(aggr_expr.as_ref())?; + + Ok(protobuf::PhysicalExprNode { + expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( + protobuf::PhysicalAggregateExprNode { + aggregate_function: Some( + physical_aggregate_expr_node::AggregateFunction::AggrFunction( + aggr_function as i32, + ), + ), + expr: expressions, + ordering_req, + distinct, + }, + )), + }) +} - let mut args = window_expr.expressions().to_vec(); - let window_frame = window_expr.get_window_frame(); +pub fn serialize_physical_window_expr( + window_expr: Arc, + codec: &dyn PhysicalExtensionCodec, +) -> Result { + let expr = window_expr.as_any(); + let mut args = window_expr.expressions().to_vec(); + let window_frame = window_expr.get_window_frame(); - let window_function = if let Some(built_in_window_expr) = - expr.downcast_ref::() + let window_function = 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 built_in_fn_expr.downcast_ref::().is_some() { + protobuf::BuiltInWindowFunction::RowNumber + } else if let Some(rank_expr) = built_in_fn_expr.downcast_ref::() { + match rank_expr.get_type() { + RankType::Basic => protobuf::BuiltInWindowFunction::Rank, + RankType::Dense => protobuf::BuiltInWindowFunction::DenseRank, + RankType::Percent => protobuf::BuiltInWindowFunction::PercentRank, + } + } else if built_in_fn_expr.downcast_ref::().is_some() { + protobuf::BuiltInWindowFunction::CumeDist + } else if let Some(ntile_expr) = built_in_fn_expr.downcast_ref::() { + args.insert( + 0, + Arc::new(Literal::new(datafusion_common::ScalarValue::Int64(Some( + ntile_expr.get_n() as i64, + )))), + ); + protobuf::BuiltInWindowFunction::Ntile + } else if let Some(window_shift_expr) = + built_in_fn_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 built_in_fn_expr.downcast_ref::().is_some() { - protobuf::BuiltInWindowFunction::RowNumber - } else if let Some(rank_expr) = built_in_fn_expr.downcast_ref::() { - match rank_expr.get_type() { - RankType::Basic => protobuf::BuiltInWindowFunction::Rank, - RankType::Dense => protobuf::BuiltInWindowFunction::DenseRank, - RankType::Percent => protobuf::BuiltInWindowFunction::PercentRank, - } - } else if built_in_fn_expr.downcast_ref::().is_some() { - protobuf::BuiltInWindowFunction::CumeDist - } else if let Some(ntile_expr) = built_in_fn_expr.downcast_ref::() { - args.insert( - 0, - Arc::new(Literal::new(datafusion_common::ScalarValue::Int64(Some( - ntile_expr.get_n() as i64, - )))), - ); - protobuf::BuiltInWindowFunction::Ntile - } else if let Some(window_shift_expr) = - built_in_fn_expr.downcast_ref::() - { - args.insert( - 1, - Arc::new(Literal::new(datafusion_common::ScalarValue::Int64(Some( - window_shift_expr.get_shift_offset(), - )))), - ); - args.insert( - 2, - Arc::new(Literal::new(window_shift_expr.get_default_value())), - ); - - if window_shift_expr.get_shift_offset() >= 0 { - protobuf::BuiltInWindowFunction::Lag - } else { - protobuf::BuiltInWindowFunction::Lead - } - } else 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 - } - } + args.insert( + 1, + Arc::new(Literal::new(datafusion_common::ScalarValue::Int64(Some( + window_shift_expr.get_shift_offset(), + )))), + ); + args.insert( + 2, + Arc::new(Literal::new(window_shift_expr.get_default_value())), + ); + + if window_shift_expr.get_shift_offset() >= 0 { + protobuf::BuiltInWindowFunction::Lag } else { - return not_impl_err!("BuiltIn function not supported: {expr:?}"); - }; - - physical_window_expr_node::WindowFunction::BuiltInFunction(builtin_fn as i32) - } else if let Some(plain_aggr_window_expr) = - expr.downcast_ref::() - { - let AggrFn { inner, distinct } = aggr_expr_to_aggr_fn( - plain_aggr_window_expr.get_aggregate_expr().as_ref(), - )?; - - if distinct { - // TODO - return not_impl_err!( - "Distinct aggregate functions not supported in window expressions" - ); + protobuf::BuiltInWindowFunction::Lead } - - if !window_frame.start_bound.is_unbounded() { - return Err(DataFusionError::Internal(format!("Invalid PlainAggregateWindowExpr = {window_expr:?} with WindowFrame = {window_frame:?}"))); + } else 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::AggrFunction(inner as i32) - } else if let Some(sliding_aggr_window_expr) = - expr.downcast_ref::() - { - let AggrFn { inner, distinct } = aggr_expr_to_aggr_fn( - sliding_aggr_window_expr.get_aggregate_expr().as_ref(), - )?; - - if distinct { - // TODO - return not_impl_err!( - "Distinct aggregate functions not supported in window expressions" - ); - } + physical_window_expr_node::WindowFunction::BuiltInFunction(builtin_fn as i32) + } else if let Some(plain_aggr_window_expr) = + expr.downcast_ref::() + { + let AggrFn { inner, distinct } = + aggr_expr_to_aggr_fn(plain_aggr_window_expr.get_aggregate_expr().as_ref())?; + + if distinct { + // TODO + return not_impl_err!( + "Distinct aggregate functions not supported in window expressions" + ); + } - if window_frame.start_bound.is_unbounded() { - return Err(DataFusionError::Internal(format!("Invalid SlidingAggregateWindowExpr = {window_expr:?} with WindowFrame = {window_frame:?}"))); - } + if !window_frame.start_bound.is_unbounded() { + return Err(DataFusionError::Internal(format!("Invalid PlainAggregateWindowExpr = {window_expr:?} with WindowFrame = {window_frame:?}"))); + } - physical_window_expr_node::WindowFunction::AggrFunction(inner as i32) - } else { - return not_impl_err!("WindowExpr not supported: {window_expr:?}"); - }; - let codec = DefaultPhysicalExtensionCodec {}; - let args = serialize_physical_exprs(args, &codec)?; - let partition_by = - serialize_physical_exprs(window_expr.partition_by().to_vec(), &codec)?; + physical_window_expr_node::WindowFunction::AggrFunction(inner as i32) + } else if let Some(sliding_aggr_window_expr) = + expr.downcast_ref::() + { + let AggrFn { inner, distinct } = + aggr_expr_to_aggr_fn(sliding_aggr_window_expr.get_aggregate_expr().as_ref())?; + + if distinct { + // TODO + return not_impl_err!( + "Distinct aggregate functions not supported in window expressions" + ); + } - let order_by = - serialize_physical_sort_exprs(window_expr.order_by().to_vec(), &codec)?; + if window_frame.start_bound.is_unbounded() { + return Err(DataFusionError::Internal(format!("Invalid SlidingAggregateWindowExpr = {window_expr:?} with WindowFrame = {window_frame:?}"))); + } - let window_frame: protobuf::WindowFrame = window_frame - .as_ref() - .try_into() - .map_err(|e| DataFusionError::Internal(format!("{e}")))?; - - let name = window_expr.name().to_string(); - - Ok(protobuf::PhysicalWindowExprNode { - args, - partition_by, - order_by, - window_frame: Some(window_frame), - window_function: Some(window_function), - name, - }) - } + physical_window_expr_node::WindowFunction::AggrFunction(inner as i32) + } else { + return not_impl_err!("WindowExpr not supported: {window_expr:?}"); + }; + + let args = serialize_physical_exprs(args, codec)?; + let partition_by = + serialize_physical_exprs(window_expr.partition_by().to_vec(), codec)?; + let order_by = serialize_physical_sort_exprs(window_expr.order_by().to_vec(), codec)?; + let window_frame: protobuf::WindowFrame = window_frame + .as_ref() + .try_into() + .map_err(|e| DataFusionError::Internal(format!("{e}")))?; + + Ok(protobuf::PhysicalWindowExprNode { + args, + partition_by, + order_by, + window_frame: Some(window_frame), + window_function: Some(window_function), + name: window_expr.name().to_string(), + }) } struct AggrFn { @@ -366,7 +348,7 @@ fn aggr_expr_to_aggr_fn(expr: &dyn AggregateExpr) -> Result { pub fn serialize_physical_sort_exprs( sort_exprs: I, codec: &dyn PhysicalExtensionCodec, -) -> Result, DataFusionError> +) -> Result> where I: IntoIterator, { @@ -379,7 +361,7 @@ where pub fn serialize_physical_sort_expr( sort_expr: PhysicalSortExpr, codec: &dyn PhysicalExtensionCodec, -) -> Result { +) -> Result { let PhysicalSortExpr { expr, options } = sort_expr; let expr = serialize_physical_expr(expr, codec)?; Ok(PhysicalSortExprNode { @@ -392,7 +374,7 @@ pub fn serialize_physical_sort_expr( pub fn serialize_physical_exprs( values: I, codec: &dyn PhysicalExtensionCodec, -) -> Result, DataFusionError> +) -> Result> where I: IntoIterator>, { @@ -409,7 +391,7 @@ where pub fn serialize_physical_expr( value: Arc, codec: &dyn PhysicalExtensionCodec, -) -> Result { +) -> Result { let expr = value.as_any(); if let Some(expr) = expr.downcast_ref::() { @@ -456,7 +438,7 @@ pub fn serialize_physical_expr( .when_then_expr() .iter() .map(|(when_expr, then_expr)| { - try_parse_when_then_expr(when_expr, then_expr, codec) + serialize_when_then_expr(when_expr, then_expr, codec) }) .collect::, @@ -623,7 +605,7 @@ pub fn serialize_physical_expr( } } -fn try_parse_when_then_expr( +fn serialize_when_then_expr( when_expr: &Arc, then_expr: &Arc, codec: &dyn PhysicalExtensionCodec, @@ -637,7 +619,7 @@ fn try_parse_when_then_expr( impl TryFrom<&PartitionedFile> for protobuf::PartitionedFile { type Error = DataFusionError; - fn try_from(pf: &PartitionedFile) -> Result { + fn try_from(pf: &PartitionedFile) -> Result { let last_modified = pf.object_meta.last_modified; let last_modified_ns = last_modified.timestamp_nanos_opt().ok_or_else(|| { DataFusionError::Plan(format!( @@ -661,7 +643,7 @@ impl TryFrom<&PartitionedFile> for protobuf::PartitionedFile { impl TryFrom<&FileRange> for protobuf::FileRange { type Error = DataFusionError; - fn try_from(value: &FileRange) -> Result { + fn try_from(value: &FileRange) -> Result { Ok(protobuf::FileRange { start: value.start, end: value.end, @@ -746,61 +728,58 @@ impl From<&ColumnStatistics> for protobuf::ColumnStats { } } -impl TryFrom<&FileScanConfig> for protobuf::FileScanExecConf { - type Error = DataFusionError; - fn try_from( - conf: &FileScanConfig, - ) -> Result { - let codec = DefaultPhysicalExtensionCodec {}; - let file_groups = conf - .file_groups - .iter() - .map(|p| p.as_slice().try_into()) - .collect::, _>>()?; - - let mut output_orderings = vec![]; - for order in &conf.output_ordering { - let ordering = serialize_physical_sort_exprs(order.to_vec(), &codec)?; - output_orderings.push(ordering) - } +pub fn serialize_file_scan_config( + conf: &FileScanConfig, + codec: &dyn PhysicalExtensionCodec, +) -> Result { + let file_groups = conf + .file_groups + .iter() + .map(|p| p.as_slice().try_into()) + .collect::, _>>()?; + + let mut output_orderings = vec![]; + for order in &conf.output_ordering { + let ordering = serialize_physical_sort_exprs(order.to_vec(), codec)?; + output_orderings.push(ordering) + } - // Fields must be added to the schema so that they can persist in the protobuf - // and then they are to be removed from the schema in `parse_protobuf_file_scan_config` - let mut fields = conf - .file_schema - .fields() + // Fields must be added to the schema so that they can persist in the protobuf, + // and then they are to be removed from the schema in `parse_protobuf_file_scan_config` + let mut fields = conf + .file_schema + .fields() + .iter() + .cloned() + .collect::>(); + fields.extend(conf.table_partition_cols.iter().cloned().map(Arc::new)); + let schema = Arc::new(arrow::datatypes::Schema::new(fields.clone())); + + Ok(protobuf::FileScanExecConf { + file_groups, + statistics: Some((&conf.statistics).into()), + limit: conf.limit.map(|l| protobuf::ScanLimit { limit: l as u32 }), + projection: conf + .projection + .as_ref() + .unwrap_or(&vec![]) .iter() - .cloned() - .collect::>(); - fields.extend(conf.table_partition_cols.iter().cloned().map(Arc::new)); - let schema = Arc::new(datafusion::arrow::datatypes::Schema::new(fields.clone())); - - Ok(protobuf::FileScanExecConf { - file_groups, - statistics: Some((&conf.statistics).into()), - limit: conf.limit.map(|l| protobuf::ScanLimit { limit: l as u32 }), - projection: conf - .projection - .as_ref() - .unwrap_or(&vec![]) - .iter() - .map(|n| *n as u32) - .collect(), - schema: Some(schema.as_ref().try_into()?), - table_partition_cols: conf - .table_partition_cols - .iter() - .map(|x| x.name().clone()) - .collect::>(), - object_store_url: conf.object_store_url.to_string(), - output_ordering: output_orderings - .into_iter() - .map(|e| PhysicalSortExprNodeCollection { - physical_sort_expr_nodes: e, - }) - .collect::>(), - }) - } + .map(|n| *n as u32) + .collect(), + schema: Some(schema.as_ref().try_into()?), + table_partition_cols: conf + .table_partition_cols + .iter() + .map(|x| x.name().clone()) + .collect::>(), + object_store_url: conf.object_store_url.to_string(), + output_ordering: output_orderings + .into_iter() + .map(|e| PhysicalSortExprNodeCollection { + physical_sort_expr_nodes: e, + }) + .collect::>(), + }) } impl From for protobuf::JoinSide { @@ -812,46 +791,15 @@ impl From for protobuf::JoinSide { } } -impl TryFrom>> for protobuf::MaybeFilter { - type Error = DataFusionError; - - fn try_from(expr: Option>) -> Result { - let codec = DefaultPhysicalExtensionCodec {}; - match expr { - None => Ok(protobuf::MaybeFilter { expr: None }), - Some(expr) => Ok(protobuf::MaybeFilter { - expr: Some(serialize_physical_expr(expr, &codec)?), - }), - } - } -} - -impl TryFrom>> for protobuf::MaybePhysicalSortExprs { - type Error = DataFusionError; - - fn try_from(sort_exprs: Option>) -> Result { - match sort_exprs { - None => Ok(protobuf::MaybePhysicalSortExprs { sort_expr: vec![] }), - Some(sort_exprs) => Ok(protobuf::MaybePhysicalSortExprs { - sort_expr: sort_exprs - .into_iter() - .map(|sort_expr| sort_expr.try_into()) - .collect::>>()?, - }), - } - } -} - -impl TryFrom for protobuf::PhysicalSortExprNode { - type Error = DataFusionError; - - fn try_from(sort_expr: PhysicalSortExpr) -> std::result::Result { - let codec = DefaultPhysicalExtensionCodec {}; - Ok(PhysicalSortExprNode { - expr: Some(Box::new(serialize_physical_expr(sort_expr.expr, &codec)?)), - asc: !sort_expr.options.descending, - nulls_first: sort_expr.options.nulls_first, - }) +pub fn serialize_maybe_filter( + expr: Option>, + codec: &dyn PhysicalExtensionCodec, +) -> Result { + match expr { + None => Ok(protobuf::MaybeFilter { expr: None }), + Some(expr) => Ok(protobuf::MaybeFilter { + expr: Some(serialize_physical_expr(expr, codec)?), + }), } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index f97cfea765bf..642860d6397b 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -21,6 +21,8 @@ use std::sync::Arc; use std::vec; use arrow::csv::WriterBuilder; +use prost::Message; + use datafusion::arrow::array::ArrayRef; use datafusion::arrow::compute::kernels::sort::SortOptions; use datafusion::arrow::datatypes::{DataType, Field, IntervalUnit, Schema}; @@ -35,7 +37,7 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; -use datafusion::physical_expr::expressions::NthValueAgg; +use datafusion::physical_expr::expressions::{Count, Max, NthValueAgg}; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; use datafusion::physical_plan::aggregates::{ @@ -77,19 +79,18 @@ use datafusion_expr::{ ScalarFunctionDefinition, ScalarUDF, ScalarUDFImpl, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, }; -use datafusion_proto::physical_plan::from_proto::parse_physical_expr; -use datafusion_proto::physical_plan::to_proto::serialize_physical_expr; use datafusion_proto::physical_plan::{ AsExecutionPlan, DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, }; use datafusion_proto::protobuf; -use prost::Message; /// 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 /// lost during serde because the string representation of a plan often only shows a subset of state. fn roundtrip_test(exec_plan: Arc) -> Result<()> { - let _ = roundtrip_test_and_return(exec_plan); + let ctx = SessionContext::new(); + let codec = DefaultPhysicalExtensionCodec {}; + roundtrip_test_and_return(exec_plan, &ctx, &codec)?; Ok(()) } @@ -101,15 +102,15 @@ fn roundtrip_test(exec_plan: Arc) -> Result<()> { /// farther in tests. fn roundtrip_test_and_return( exec_plan: Arc, + ctx: &SessionContext, + codec: &dyn PhysicalExtensionCodec, ) -> Result> { - let ctx = SessionContext::new(); - let codec = DefaultPhysicalExtensionCodec {}; let proto: protobuf::PhysicalPlanNode = - protobuf::PhysicalPlanNode::try_from_physical_plan(exec_plan.clone(), &codec) + protobuf::PhysicalPlanNode::try_from_physical_plan(exec_plan.clone(), codec) .expect("to proto"); let runtime = ctx.runtime_env(); let result_exec_plan: Arc = proto - .try_into_physical_plan(&ctx, runtime.deref(), &codec) + .try_into_physical_plan(ctx, runtime.deref(), codec) .expect("from proto"); assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); Ok(result_exec_plan) @@ -123,17 +124,10 @@ fn roundtrip_test_and_return( /// performing serde on some plans. fn roundtrip_test_with_context( exec_plan: Arc, - ctx: SessionContext, + ctx: &SessionContext, ) -> Result<()> { let codec = DefaultPhysicalExtensionCodec {}; - let proto: protobuf::PhysicalPlanNode = - protobuf::PhysicalPlanNode::try_from_physical_plan(exec_plan.clone(), &codec) - .expect("to proto"); - let runtime = ctx.runtime_env(); - let result_exec_plan: Arc = proto - .try_into_physical_plan(&ctx, runtime.deref(), &codec) - .expect("from proto"); - assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); + roundtrip_test_and_return(exec_plan, ctx, &codec)?; Ok(()) } @@ -444,7 +438,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { Arc::new(EmptyExec::new(schema.clone())), schema, )?), - ctx, + &ctx, ) } @@ -642,11 +636,11 @@ fn roundtrip_scalar_udf() -> Result<()> { ctx.register_udf(udf); - roundtrip_test_with_context(Arc::new(project), ctx) + roundtrip_test_with_context(Arc::new(project), &ctx) } #[test] -fn roundtrip_scalar_udf_extension_codec() { +fn roundtrip_scalar_udf_extension_codec() -> Result<()> { #[derive(Debug)] struct MyRegexUdf { signature: Signature, @@ -657,11 +651,7 @@ fn roundtrip_scalar_udf_extension_codec() { impl MyRegexUdf { fn new(pattern: String) -> Self { Self { - signature: Signature::uniform( - 1, - vec![DataType::Int32], - Volatility::Immutable, - ), + signature: Signature::exact(vec![DataType::Utf8], Volatility::Immutable), pattern, } } @@ -672,18 +662,22 @@ fn roundtrip_scalar_udf_extension_codec() { fn as_any(&self) -> &dyn Any { self } + fn name(&self) -> &str { "regex_udf" } + fn signature(&self) -> &Signature { &self.signature } + fn return_type(&self, args: &[DataType]) -> Result { if !matches!(args.first(), Some(&DataType::Utf8)) { return plan_err!("regex_udf only accepts Utf8 arguments"); } - Ok(DataType::Int32) + Ok(DataType::Int64) } + fn invoke(&self, _args: &[ColumnarValue]) -> Result { unimplemented!() } @@ -747,32 +741,58 @@ fn roundtrip_scalar_udf_extension_codec() { } } + let field_text = Field::new("text", DataType::Utf8, true); + let field_published = Field::new("published", DataType::Boolean, false); + let field_author = Field::new("author", DataType::Utf8, false); + let schema = Arc::new(Schema::new(vec![field_text, field_published, field_author])); + let input = Arc::new(EmptyExec::new(schema.clone())); + let pattern = ".*"; let udf = ScalarUDF::from(MyRegexUdf::new(pattern.to_string())); - let test_expr = ScalarFunctionExpr::new( + let udf_expr = Arc::new(ScalarFunctionExpr::new( udf.name(), ScalarFunctionDefinition::UDF(Arc::new(udf.clone())), - vec![], - DataType::Int32, + vec![col("text", &schema)?], + DataType::Int64, None, false, - ); - let fmt_expr = format!("{test_expr:?}"); - let ctx = SessionContext::new(); + )); - ctx.register_udf(udf.clone()); - let extension_codec = ScalarUDFExtensionCodec {}; - let proto: protobuf::PhysicalExprNode = - match serialize_physical_expr(Arc::new(test_expr), &extension_codec) { - Ok(proto) => proto, - Err(e) => panic!("failed to serialize expr: {e:?}"), - }; - let field_a = Field::new("a", DataType::Int32, false); - let schema = Arc::new(Schema::new(vec![field_a])); - let round_trip = - parse_physical_expr(&proto, &ctx, &schema, &extension_codec).unwrap(); - assert_eq!(fmt_expr, format!("{round_trip:?}")); + let filter = Arc::new(FilterExec::try_new( + Arc::new(BinaryExpr::new( + col("published", &schema)?, + Operator::And, + Arc::new(BinaryExpr::new(udf_expr.clone(), Operator::Gt, lit(0))), + )), + input, + )?); + + let window = Arc::new(WindowAggExec::try_new( + vec![Arc::new(PlainAggregateWindowExpr::new( + Arc::new(Max::new(udf_expr.clone(), "max", DataType::Int64)), + &[col("author", &schema)?], + &[], + Arc::new(WindowFrame::new(None)), + ))], + filter, + vec![col("author", &schema)?], + )?); + + let aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::new(vec![], vec![], vec![]), + vec![Arc::new(Count::new(udf_expr, "count", DataType::Int64))], + vec![None], + window, + schema.clone(), + )?); + + let ctx = SessionContext::new(); + let codec = ScalarUDFExtensionCodec {}; + roundtrip_test_and_return(aggregate, &ctx, &codec)?; + Ok(()) } + #[test] fn roundtrip_distinct_count() -> Result<()> { let field_a = Field::new("a", DataType::Int64, false); @@ -896,12 +916,18 @@ fn roundtrip_csv_sink() -> Result<()> { }), )]; - let roundtrip_plan = roundtrip_test_and_return(Arc::new(DataSinkExec::new( - input, - data_sink, - schema.clone(), - Some(sort_order), - ))) + let ctx = SessionContext::new(); + let codec = DefaultPhysicalExtensionCodec {}; + let roundtrip_plan = roundtrip_test_and_return( + Arc::new(DataSinkExec::new( + input, + data_sink, + schema.clone(), + Some(sort_order), + )), + &ctx, + &codec, + ) .unwrap(); let roundtrip_plan = roundtrip_plan From 635faa6c85f1dd5f739438e4ae569c1639c1332a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 15 Apr 2024 06:28:46 -0400 Subject: [PATCH 34/39] Minor: Do not truncate `SHOW ALL` in datafusion-cli (#10079) --- datafusion-cli/src/exec.rs | 67 +++++++++++++++++++++++++++++--------- 1 file changed, 52 insertions(+), 15 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 2072cc7df002..5fbcea0c0683 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -42,6 +42,7 @@ use datafusion::sql::parser::{DFParser, Statement}; use datafusion::sql::sqlparser::dialect::dialect_from_str; use datafusion::common::FileType; +use datafusion::sql::sqlparser; use rustyline::error::ReadlineError; use rustyline::Editor; use tokio::signal; @@ -221,15 +222,12 @@ async fn exec_and_print( let statements = DFParser::parse_sql_with_dialect(&sql, dialect.as_ref())?; for statement in statements { + let adjusted = + AdjustedPrintOptions::new(print_options.clone()).with_statement(&statement); + let plan = create_plan(ctx, statement).await?; + let adjusted = adjusted.with_plan(&plan); - // For plans like `Explain` ignore `MaxRows` option and always display all rows - let should_ignore_maxrows = matches!( - plan, - LogicalPlan::Explain(_) - | LogicalPlan::DescribeTable(_) - | LogicalPlan::Analyze(_) - ); let df = ctx.execute_logical_plan(plan).await?; let physical_plan = df.create_physical_plan().await?; @@ -237,21 +235,60 @@ async fn exec_and_print( let stream = execute_stream(physical_plan, task_ctx.clone())?; print_options.print_stream(stream, now).await?; } else { - let mut print_options = print_options.clone(); - if should_ignore_maxrows { - print_options.maxrows = MaxRows::Unlimited; - } - if print_options.format == PrintFormat::Automatic { - print_options.format = PrintFormat::Table; - } let results = collect(physical_plan, task_ctx.clone()).await?; - print_options.print_batches(&results, now)?; + adjusted.into_inner().print_batches(&results, now)?; } } Ok(()) } +/// Track adjustments to the print options based on the plan / statement being executed +#[derive(Debug)] +struct AdjustedPrintOptions { + inner: PrintOptions, +} + +impl AdjustedPrintOptions { + fn new(inner: PrintOptions) -> Self { + Self { inner } + } + /// Adjust print options based on any statement specific requirements + fn with_statement(mut self, statement: &Statement) -> Self { + if let Statement::Statement(sql_stmt) = statement { + // SHOW / SHOW ALL + if let sqlparser::ast::Statement::ShowVariable { .. } = sql_stmt.as_ref() { + self.inner.maxrows = MaxRows::Unlimited + } + } + self + } + + /// Adjust print options based on any plan specific requirements + fn with_plan(mut self, plan: &LogicalPlan) -> Self { + // For plans like `Explain` ignore `MaxRows` option and always display + // all rows + if matches!( + plan, + LogicalPlan::Explain(_) + | LogicalPlan::DescribeTable(_) + | LogicalPlan::Analyze(_) + ) { + self.inner.maxrows = MaxRows::Unlimited; + } + self + } + + /// Finalize and return the inner `PrintOptions` + fn into_inner(mut self) -> PrintOptions { + if self.inner.format == PrintFormat::Automatic { + self.inner.format = PrintFormat::Table; + } + + self.inner + } +} + async fn create_plan( ctx: &mut SessionContext, statement: Statement, From 8604d0a80155364ff60d3eac5e44f03f6e7e5a8e Mon Sep 17 00:00:00 2001 From: LFC <990479+MichaelScofield@users.noreply.github.com> Date: Mon, 15 Apr 2024 18:55:53 +0800 Subject: [PATCH 35/39] Minor: get mutable ref to `SessionConfig` in `SessionState` (#10050) * Get mutable `SessionConfig` in `SessionState` so that we can add extensions to it. * fix doc tests * fix: resolve PR comments --- datafusion/core/src/execution/context/mod.rs | 5 +++ datafusion/execution/src/config.rs | 43 +++++++++++++++++++- 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 5cf8969aa46d..fc2cdbb7518d 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1952,6 +1952,11 @@ impl SessionState { &self.config } + /// Return the mutable [`SessionConfig`]. + pub fn config_mut(&mut self) -> &mut SessionConfig { + &mut self.config + } + /// Return the physical optimizers pub fn physical_optimizers(&self) -> &[Arc] { &self.physical_optimizers.rules diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 0a7a87c7d81a..e29030e61457 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -501,13 +501,54 @@ impl SessionConfig { /// /// [^1]: Compare that to [`ConfigOptions`] which only supports [`ScalarValue`] payloads. pub fn with_extension(mut self, ext: Arc) -> Self + where + T: Send + Sync + 'static, + { + self.set_extension(ext); + self + } + + /// Set extension. Pretty much the same as [`with_extension`](Self::with_extension), but take + /// mutable reference instead of owning it. Useful if you want to add another extension after + /// the [`SessionConfig`] is created. + /// + /// # Example + /// ``` + /// use std::sync::Arc; + /// use datafusion_execution::config::SessionConfig; + /// + /// // application-specific extension types + /// struct Ext1(u8); + /// struct Ext2(u8); + /// struct Ext3(u8); + /// + /// let ext1a = Arc::new(Ext1(10)); + /// let ext1b = Arc::new(Ext1(11)); + /// let ext2 = Arc::new(Ext2(2)); + /// + /// let mut cfg = SessionConfig::default(); + /// + /// // will only remember the last Ext1 + /// cfg.set_extension(Arc::clone(&ext1a)); + /// cfg.set_extension(Arc::clone(&ext1b)); + /// cfg.set_extension(Arc::clone(&ext2)); + /// + /// let ext1_received = cfg.get_extension::().unwrap(); + /// assert!(!Arc::ptr_eq(&ext1_received, &ext1a)); + /// assert!(Arc::ptr_eq(&ext1_received, &ext1b)); + /// + /// let ext2_received = cfg.get_extension::().unwrap(); + /// assert!(Arc::ptr_eq(&ext2_received, &ext2)); + /// + /// assert!(cfg.get_extension::().is_none()); + /// ``` + pub fn set_extension(&mut self, ext: Arc) where T: Send + Sync + 'static, { let ext = ext as Arc; let id = TypeId::of::(); self.extensions.insert(id, ext); - self } /// Get extension, if any for the specified type `T` exists. From beeb80830bc3aab01f92e3ac568055510bdbe382 Mon Sep 17 00:00:00 2001 From: Eren Avsarogullari Date: Mon, 15 Apr 2024 03:56:38 -0700 Subject: [PATCH 36/39] Move `ceil`, `exp`, `factorial` to `datafusion-functions` crate (#10083) * Issue-9939 - Move ceil, exp, factorial to datafusion-functions crate * Issue-9939 - Fix build failures * Update Volatility --------- Co-authored-by: Andrew Lamb --- datafusion/expr/src/built_in_function.rs | 44 +------ datafusion/expr/src/expr_fn.rs | 30 ----- datafusion/functions/src/math/factorial.rs | 117 ++++++++++++++++++ datafusion/functions/src/math/mod.rs | 22 ++++ .../physical-expr/src/equivalence/ordering.rs | 11 +- datafusion/physical-expr/src/functions.rs | 54 +------- .../physical-expr/src/math_expressions.rs | 107 +--------------- datafusion/proto/proto/datafusion.proto | 6 +- datafusion/proto/src/generated/pbjson.rs | 9 -- datafusion/proto/src/generated/prost.rs | 12 +- .../proto/src/logical_plan/from_proto.rs | 12 +- datafusion/proto/src/logical_plan/to_proto.rs | 3 - datafusion/sql/src/expr/function.rs | 11 -- datafusion/sql/src/expr/mod.rs | 12 +- 14 files changed, 161 insertions(+), 289 deletions(-) create mode 100644 datafusion/functions/src/math/factorial.rs diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 43cb0c3e0a50..5bfec00ea3b3 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -37,14 +37,8 @@ use strum_macros::EnumIter; #[derive(Debug, Clone, PartialEq, Eq, Hash, EnumIter, Copy)] pub enum BuiltinScalarFunction { // math functions - /// ceil - Ceil, /// coalesce Coalesce, - /// exp - Exp, - /// factorial - Factorial, // string functions /// concat Concat, @@ -106,10 +100,7 @@ impl BuiltinScalarFunction { pub fn volatility(&self) -> Volatility { match self { // Immutable scalar builtins - BuiltinScalarFunction::Ceil => Volatility::Immutable, BuiltinScalarFunction::Coalesce => Volatility::Immutable, - BuiltinScalarFunction::Exp => Volatility::Immutable, - BuiltinScalarFunction::Factorial => Volatility::Immutable, BuiltinScalarFunction::Concat => Volatility::Immutable, BuiltinScalarFunction::ConcatWithSeparator => Volatility::Immutable, BuiltinScalarFunction::EndsWith => Volatility::Immutable, @@ -145,15 +136,6 @@ impl BuiltinScalarFunction { utf8_to_str_type(&input_expr_types[0], "initcap") } BuiltinScalarFunction::EndsWith => Ok(Boolean), - - BuiltinScalarFunction::Factorial => Ok(Int64), - - BuiltinScalarFunction::Ceil | BuiltinScalarFunction::Exp => { - match input_expr_types[0] { - Float32 => Ok(Float32), - _ => Ok(Float64), - } - } } } @@ -185,17 +167,6 @@ impl BuiltinScalarFunction { ], self.volatility(), ), - BuiltinScalarFunction::Factorial => { - Signature::uniform(1, vec![Int64], self.volatility()) - } - BuiltinScalarFunction::Ceil | BuiltinScalarFunction::Exp => { - // math expressions expect 1 argument of type f64 or f32 - // priority is given to f64 because e.g. `sqrt(1i32)` is in IR (real numbers) and thus we - // return the best approximation for it (in f64). - // We accept f32 because in this case it is clear that the best approximation - // will be as good as the number of digits in the number - Signature::uniform(1, vec![Float64, Float32], self.volatility()) - } } } @@ -203,25 +174,12 @@ impl BuiltinScalarFunction { /// The list can be extended, only mathematical and datetime functions are /// considered for the initial implementation of this feature. pub fn monotonicity(&self) -> Option { - if matches!( - &self, - BuiltinScalarFunction::Ceil - | BuiltinScalarFunction::Exp - | BuiltinScalarFunction::Factorial - ) { - Some(vec![Some(true)]) - } else { - None - } + None } /// Returns all names that can be used to call this function pub fn aliases(&self) -> &'static [&'static str] { match self { - BuiltinScalarFunction::Ceil => &["ceil"], - BuiltinScalarFunction::Exp => &["exp"], - BuiltinScalarFunction::Factorial => &["factorial"], - // conditional functions BuiltinScalarFunction::Coalesce => &["coalesce"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 6a28275ebfcf..f7900f6b197d 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -525,16 +525,6 @@ macro_rules! nary_scalar_expr { // generate methods for creating the supported unary/binary expressions // math functions -scalar_expr!(Factorial, factorial, num, "factorial"); -scalar_expr!( - Ceil, - ceil, - num, - "nearest integer greater than or equal to argument" -); - -scalar_expr!(Exp, exp, num, "exponential"); - scalar_expr!(InitCap, initcap, string, "converts the first letter of each word in `string` in uppercase and the remaining characters in lowercase"); scalar_expr!(EndsWith, ends_with, string suffix, "whether the `string` ends with the `suffix`"); nary_scalar_expr!(Coalesce, coalesce, "returns `coalesce(args...)`, which evaluates to the value of the first [Expr] which is not NULL"); @@ -877,22 +867,6 @@ mod test { ); } - macro_rules! test_unary_scalar_expr { - ($ENUM:ident, $FUNC:ident) => {{ - if let Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn(fun), - args, - }) = $FUNC(col("tableA.a")) - { - let name = built_in_function::BuiltinScalarFunction::$ENUM; - assert_eq!(name, fun); - assert_eq!(1, args.len()); - } else { - assert!(false, "unexpected"); - } - }}; - } - macro_rules! test_scalar_expr { ($ENUM:ident, $FUNC:ident, $($arg:ident),*) => { let expected = [$(stringify!($arg)),*]; @@ -913,10 +887,6 @@ mod test { #[test] fn scalar_function_definitions() { - test_unary_scalar_expr!(Factorial, factorial); - test_unary_scalar_expr!(Ceil, ceil); - test_unary_scalar_expr!(Exp, exp); - test_scalar_expr!(InitCap, initcap, string); test_scalar_expr!(EndsWith, ends_with, string, characters); } diff --git a/datafusion/functions/src/math/factorial.rs b/datafusion/functions/src/math/factorial.rs new file mode 100644 index 000000000000..dc481da79069 --- /dev/null +++ b/datafusion/functions/src/math/factorial.rs @@ -0,0 +1,117 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{ArrayRef, Int64Array}; +use std::any::Any; +use std::sync::Arc; + +use arrow::datatypes::DataType; +use arrow::datatypes::DataType::Int64; + +use crate::utils::make_scalar_function; +use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; + +#[derive(Debug)] +pub struct FactorialFunc { + signature: Signature, +} + +impl Default for FactorialFunc { + fn default() -> Self { + FactorialFunc::new() + } +} + +impl FactorialFunc { + pub fn new() -> Self { + Self { + signature: Signature::uniform(1, vec![Int64], Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for FactorialFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "factorial" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(Int64) + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + make_scalar_function(factorial, vec![])(args) + } +} + +macro_rules! make_function_scalar_inputs { + ($ARG: expr, $NAME:expr, $ARRAY_TYPE:ident, $FUNC: block) => {{ + let arg = downcast_arg!($ARG, $NAME, $ARRAY_TYPE); + + arg.iter() + .map(|a| match a { + Some(a) => Some($FUNC(a)), + _ => None, + }) + .collect::<$ARRAY_TYPE>() + }}; +} + +/// Factorial SQL function +fn factorial(args: &[ArrayRef]) -> Result { + match args[0].data_type() { + DataType::Int64 => Ok(Arc::new(make_function_scalar_inputs!( + &args[0], + "value", + Int64Array, + { |value: i64| { (1..=value).product() } } + )) as ArrayRef), + other => exec_err!("Unsupported data type {other:?} for function factorial."), + } +} + +#[cfg(test)] +mod test { + + use datafusion_common::cast::as_int64_array; + + use super::*; + + #[test] + fn test_factorial_i64() { + let args: Vec = vec![ + Arc::new(Int64Array::from(vec![0, 1, 2, 4])), // input + ]; + + let result = factorial(&args).expect("failed to initialize function factorial"); + let ints = + as_int64_array(&result).expect("failed to initialize function factorial"); + + let expected = Int64Array::from(vec![1, 1, 2, 24]); + + assert_eq!(ints, &expected); + } +} diff --git a/datafusion/functions/src/math/mod.rs b/datafusion/functions/src/math/mod.rs index c83a98cb1913..b6e8d26b6460 100644 --- a/datafusion/functions/src/math/mod.rs +++ b/datafusion/functions/src/math/mod.rs @@ -22,6 +22,7 @@ use std::sync::Arc; pub mod abs; pub mod cot; +pub mod factorial; pub mod gcd; pub mod iszero; pub mod lcm; @@ -44,10 +45,13 @@ make_math_unary_udf!(AtanFunc, ATAN, atan, atan, Some(vec![Some(true)])); make_math_unary_udf!(AtanhFunc, ATANH, atanh, atanh, Some(vec![Some(true)])); make_math_binary_udf!(Atan2, ATAN2, atan2, atan2, Some(vec![Some(true)])); make_math_unary_udf!(CbrtFunc, CBRT, cbrt, cbrt, None); +make_math_unary_udf!(CeilFunc, CEIL, ceil, ceil, Some(vec![Some(true)])); make_math_unary_udf!(CosFunc, COS, cos, cos, None); make_math_unary_udf!(CoshFunc, COSH, cosh, cosh, None); make_udf_function!(cot::CotFunc, COT, cot); make_math_unary_udf!(DegreesFunc, DEGREES, degrees, to_degrees, None); +make_math_unary_udf!(ExpFunc, EXP, exp, exp, Some(vec![Some(true)])); +make_udf_function!(factorial::FactorialFunc, FACTORIAL, factorial); make_math_unary_udf!(FloorFunc, FLOOR, floor, floor, Some(vec![Some(true)])); make_udf_function!(log::LogFunc, LOG, log); make_udf_function!(gcd::GcdFunc, GCD, gcd); @@ -119,6 +123,11 @@ pub mod expr_fn { super::cbrt().call(vec![num]) } + #[doc = "nearest integer greater than or equal to argument"] + pub fn ceil(num: Expr) -> Expr { + super::ceil().call(vec![num]) + } + #[doc = "cosine"] pub fn cos(num: Expr) -> Expr { super::cos().call(vec![num]) @@ -139,6 +148,16 @@ pub mod expr_fn { super::degrees().call(vec![num]) } + #[doc = "exponential"] + pub fn exp(num: Expr) -> Expr { + super::exp().call(vec![num]) + } + + #[doc = "factorial"] + pub fn factorial(num: Expr) -> Expr { + super::factorial().call(vec![num]) + } + #[doc = "nearest integer less than or equal to argument"] pub fn floor(num: Expr) -> Expr { super::floor().call(vec![num]) @@ -262,10 +281,13 @@ pub fn functions() -> Vec> { atan2(), atanh(), cbrt(), + ceil(), cos(), cosh(), cot(), degrees(), + exp(), + factorial(), floor(), gcd(), isnan(), diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 688cdf798bdd..ed4600f2d95e 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -228,8 +228,7 @@ mod tests { use itertools::Itertools; use datafusion_common::{DFSchema, Result}; - use datafusion_expr::execution_props::ExecutionProps; - use datafusion_expr::{BuiltinScalarFunction, Operator, ScalarUDF}; + use datafusion_expr::{Operator, ScalarUDF}; use crate::equivalence::tests::{ convert_to_orderings, convert_to_sort_exprs, create_random_schema, @@ -241,7 +240,6 @@ mod tests { }; use crate::expressions::Column; use crate::expressions::{col, BinaryExpr}; - use crate::functions::create_physical_expr; use crate::utils::tests::TestScalarUDF; use crate::{PhysicalExpr, PhysicalSortExpr}; @@ -301,11 +299,12 @@ mod tests { &[], &DFSchema::empty(), )?; - let exp_a = &create_physical_expr( - &BuiltinScalarFunction::Exp, + let exp_a = &crate::udf::create_physical_expr( + &test_fun, &[col("a", &test_schema)?], &test_schema, - &ExecutionProps::default(), + &[], + &DFSchema::empty(), )?; let a_plus_b = Arc::new(BinaryExpr::new( col_a.clone(), diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index c237e2070675..6efbc4179ff4 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -50,8 +50,7 @@ use datafusion_expr::{ use crate::sort_properties::SortProperties; use crate::{ - conditional_expressions, math_expressions, string_expressions, PhysicalExpr, - ScalarFunctionExpr, + conditional_expressions, string_expressions, PhysicalExpr, ScalarFunctionExpr, }; /// Create a physical (function) expression. @@ -178,12 +177,6 @@ pub fn create_physical_fun( fun: &BuiltinScalarFunction, ) -> Result { Ok(match fun { - // math functions - BuiltinScalarFunction::Ceil => Arc::new(math_expressions::ceil), - BuiltinScalarFunction::Exp => Arc::new(math_expressions::exp), - BuiltinScalarFunction::Factorial => { - Arc::new(|args| make_scalar_function_inner(math_expressions::factorial)(args)) - } // string functions BuiltinScalarFunction::Coalesce => Arc::new(conditional_expressions::coalesce), BuiltinScalarFunction::Concat => Arc::new(string_expressions::concat), @@ -279,10 +272,7 @@ fn func_order_in_one_dimension( #[cfg(test)] mod tests { use arrow::{ - array::{ - Array, ArrayRef, BooleanArray, Float32Array, Float64Array, Int32Array, - StringArray, UInt64Array, - }, + array::{Array, ArrayRef, BooleanArray, Int32Array, StringArray, UInt64Array}, datatypes::Field, record_batch::RecordBatch, }; @@ -410,46 +400,6 @@ mod tests { Utf8, StringArray ); - test_function!( - Exp, - &[lit(ScalarValue::Int32(Some(1)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::UInt32(Some(1)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::UInt64(Some(1)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::Float64(Some(1.0)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::Float32(Some(1.0)))], - Ok(Some((1.0_f32).exp())), - f32, - Float32, - Float32Array - ); test_function!( InitCap, &[lit("hi THOMAS")], diff --git a/datafusion/physical-expr/src/math_expressions.rs b/datafusion/physical-expr/src/math_expressions.rs index 004a9abe7f0b..cee1b8c787e2 100644 --- a/datafusion/physical-expr/src/math_expressions.rs +++ b/datafusion/physical-expr/src/math_expressions.rs @@ -21,69 +21,12 @@ use std::any::type_name; use std::sync::Arc; use arrow::array::ArrayRef; -use arrow::array::{BooleanArray, Float32Array, Float64Array, Int64Array}; +use arrow::array::{BooleanArray, Float32Array, Float64Array}; use arrow::datatypes::DataType; use arrow_array::Array; -use datafusion_common::{exec_err, ScalarValue}; +use datafusion_common::exec_err; use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::ColumnarValue; - -macro_rules! downcast_compute_op { - ($ARRAY:expr, $NAME:expr, $FUNC:ident, $TYPE:ident) => {{ - let n = $ARRAY.as_any().downcast_ref::<$TYPE>(); - match n { - Some(array) => { - let res: $TYPE = - arrow::compute::kernels::arity::unary(array, |x| x.$FUNC()); - Ok(Arc::new(res)) - } - _ => exec_err!("Invalid data type for {}", $NAME), - } - }}; -} - -macro_rules! unary_primitive_array_op { - ($VALUE:expr, $NAME:expr, $FUNC:ident) => {{ - match ($VALUE) { - ColumnarValue::Array(array) => match array.data_type() { - DataType::Float32 => { - let result = downcast_compute_op!(array, $NAME, $FUNC, Float32Array); - Ok(ColumnarValue::Array(result?)) - } - DataType::Float64 => { - let result = downcast_compute_op!(array, $NAME, $FUNC, Float64Array); - Ok(ColumnarValue::Array(result?)) - } - other => { - exec_err!("Unsupported data type {:?} for function {}", other, $NAME) - } - }, - ColumnarValue::Scalar(a) => match a { - ScalarValue::Float32(a) => Ok(ColumnarValue::Scalar( - ScalarValue::Float32(a.map(|x| x.$FUNC())), - )), - ScalarValue::Float64(a) => Ok(ColumnarValue::Scalar( - ScalarValue::Float64(a.map(|x| x.$FUNC())), - )), - _ => exec_err!( - "Unsupported data type {:?} for function {}", - ($VALUE).data_type(), - $NAME - ), - }, - } - }}; -} - -macro_rules! math_unary_function { - ($NAME:expr, $FUNC:ident) => { - /// mathematical function that accepts f32 or f64 and returns f64 - pub fn $FUNC(args: &[ColumnarValue]) -> Result { - unary_primitive_array_op!(&args[0], $NAME, $FUNC) - } - }; -} macro_rules! downcast_arg { ($ARG:expr, $NAME:expr, $ARRAY_TYPE:ident) => {{ @@ -98,19 +41,6 @@ macro_rules! downcast_arg { }}; } -macro_rules! make_function_scalar_inputs { - ($ARG: expr, $NAME:expr, $ARRAY_TYPE:ident, $FUNC: block) => {{ - let arg = downcast_arg!($ARG, $NAME, $ARRAY_TYPE); - - arg.iter() - .map(|a| match a { - Some(a) => Some($FUNC(a)), - _ => None, - }) - .collect::<$ARRAY_TYPE>() - }}; -} - macro_rules! make_function_scalar_inputs_return_type { ($ARG: expr, $NAME:expr, $ARGS_TYPE:ident, $RETURN_TYPE:ident, $FUNC: block) => {{ let arg = downcast_arg!($ARG, $NAME, $ARGS_TYPE); @@ -124,22 +54,6 @@ macro_rules! make_function_scalar_inputs_return_type { }}; } -math_unary_function!("ceil", ceil); -math_unary_function!("exp", exp); - -/// Factorial SQL function -pub fn factorial(args: &[ArrayRef]) -> Result { - match args[0].data_type() { - DataType::Int64 => Ok(Arc::new(make_function_scalar_inputs!( - &args[0], - "value", - Int64Array, - { |value: i64| { (1..=value).product() } } - )) as ArrayRef), - other => exec_err!("Unsupported data type {other:?} for function factorial."), - } -} - /// Isnan SQL function pub fn isnan(args: &[ArrayRef]) -> Result { match args[0].data_type() { @@ -167,25 +81,10 @@ pub fn isnan(args: &[ArrayRef]) -> Result { mod tests { use arrow::array::Float64Array; - use datafusion_common::cast::{as_boolean_array, as_int64_array}; + use datafusion_common::cast::as_boolean_array; use super::*; - #[test] - fn test_factorial_i64() { - let args: Vec = vec![ - Arc::new(Int64Array::from(vec![0, 1, 2, 4])), // input - ]; - - let result = factorial(&args).expect("failed to initialize function factorial"); - let ints = - as_int64_array(&result).expect("failed to initialize function factorial"); - - let expected = Int64Array::from(vec![1, 1, 2, 24]); - - assert_eq!(ints, &expected); - } - #[test] fn test_isnan_f64() { let args: Vec = vec![Arc::new(Float64Array::from(vec![ diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index e1bcf33b8254..6578c64cff1f 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -546,10 +546,10 @@ enum ScalarFunction { // 2 was Asin // 3 was Atan // 4 was Ascii - Ceil = 5; + // 5 was Ceil // 6 was Cos // 7 was Digest - Exp = 8; + // 8 was Exp // 9 was Floor // 10 was Ln // 11 was Log @@ -624,7 +624,7 @@ enum ScalarFunction { // 80 was Pi // 81 was Degrees // 82 was Radians - Factorial = 83; + // 83 was Factorial // 84 was Lcm // 85 was Gcd // 86 was ArrayAppend diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 7beaeef0e58b..1546d75f2acd 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -22792,13 +22792,10 @@ impl serde::Serialize for ScalarFunction { { let variant = match self { Self::Unknown => "unknown", - Self::Ceil => "Ceil", - Self::Exp => "Exp", Self::Concat => "Concat", Self::ConcatWithSeparator => "ConcatWithSeparator", Self::InitCap => "InitCap", Self::Coalesce => "Coalesce", - Self::Factorial => "Factorial", Self::EndsWith => "EndsWith", }; serializer.serialize_str(variant) @@ -22812,13 +22809,10 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { { const FIELDS: &[&str] = &[ "unknown", - "Ceil", - "Exp", "Concat", "ConcatWithSeparator", "InitCap", "Coalesce", - "Factorial", "EndsWith", ]; @@ -22861,13 +22855,10 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { { match value { "unknown" => Ok(ScalarFunction::Unknown), - "Ceil" => Ok(ScalarFunction::Ceil), - "Exp" => Ok(ScalarFunction::Exp), "Concat" => Ok(ScalarFunction::Concat), "ConcatWithSeparator" => Ok(ScalarFunction::ConcatWithSeparator), "InitCap" => Ok(ScalarFunction::InitCap), "Coalesce" => Ok(ScalarFunction::Coalesce), - "Factorial" => Ok(ScalarFunction::Factorial), "EndsWith" => Ok(ScalarFunction::EndsWith), _ => 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 d6a27dbc5652..c752743cbdce 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2846,10 +2846,10 @@ pub enum ScalarFunction { /// 2 was Asin /// 3 was Atan /// 4 was Ascii - Ceil = 5, + /// 5 was Ceil /// 6 was Cos /// 7 was Digest - Exp = 8, + /// 8 was Exp /// 9 was Floor /// 10 was Ln /// 11 was Log @@ -2924,7 +2924,7 @@ pub enum ScalarFunction { /// 80 was Pi /// 81 was Degrees /// 82 was Radians - Factorial = 83, + /// 83 was Factorial /// 84 was Lcm /// 85 was Gcd /// 86 was ArrayAppend @@ -2988,13 +2988,10 @@ impl ScalarFunction { pub fn as_str_name(&self) -> &'static str { match self { ScalarFunction::Unknown => "unknown", - ScalarFunction::Ceil => "Ceil", - ScalarFunction::Exp => "Exp", ScalarFunction::Concat => "Concat", ScalarFunction::ConcatWithSeparator => "ConcatWithSeparator", ScalarFunction::InitCap => "InitCap", ScalarFunction::Coalesce => "Coalesce", - ScalarFunction::Factorial => "Factorial", ScalarFunction::EndsWith => "EndsWith", } } @@ -3002,13 +2999,10 @@ impl ScalarFunction { pub fn from_str_name(value: &str) -> ::core::option::Option { match value { "unknown" => Some(Self::Unknown), - "Ceil" => Some(Self::Ceil), - "Exp" => Some(Self::Exp), "Concat" => Some(Self::Concat), "ConcatWithSeparator" => Some(Self::ConcatWithSeparator), "InitCap" => Some(Self::InitCap), "Coalesce" => Some(Self::Coalesce), - "Factorial" => Some(Self::Factorial), "EndsWith" => Some(Self::EndsWith), _ => None, } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 057690aacee6..4e61385bb545 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -37,9 +37,9 @@ use datafusion_expr::expr::Unnest; use datafusion_expr::expr::{Alias, Placeholder}; use datafusion_expr::window_frame::{check_window_frame, regularize_window_order_by}; use datafusion_expr::{ - ceil, coalesce, concat_expr, concat_ws_expr, ends_with, exp, + coalesce, concat_expr, concat_ws_expr, ends_with, expr::{self, InList, Sort, WindowFunction}, - factorial, initcap, + initcap, logical_plan::{PlanType, StringifiedPlan}, AggregateFunction, Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, @@ -418,9 +418,6 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { use protobuf::ScalarFunction; match f { ScalarFunction::Unknown => todo!(), - ScalarFunction::Exp => Self::Exp, - ScalarFunction::Factorial => Self::Factorial, - ScalarFunction::Ceil => Self::Ceil, ScalarFunction::Concat => Self::Concat, ScalarFunction::ConcatWithSeparator => Self::ConcatWithSeparator, ScalarFunction::EndsWith => Self::EndsWith, @@ -1287,11 +1284,6 @@ pub fn parse_expr( match scalar_function { ScalarFunction::Unknown => Err(proto_error("Unknown scalar function")), - ScalarFunction::Exp => Ok(exp(parse_expr(&args[0], registry, codec)?)), - ScalarFunction::Factorial => { - Ok(factorial(parse_expr(&args[0], registry, codec)?)) - } - ScalarFunction::Ceil => Ok(ceil(parse_expr(&args[0], registry, codec)?)), ScalarFunction::InitCap => { Ok(initcap(parse_expr(&args[0], registry, codec)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 358eea785713..2680bc15e1b4 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1407,9 +1407,6 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { fn try_from(scalar: &BuiltinScalarFunction) -> Result { let scalar_function = match scalar { - BuiltinScalarFunction::Exp => Self::Exp, - BuiltinScalarFunction::Factorial => Self::Factorial, - BuiltinScalarFunction::Ceil => Self::Ceil, BuiltinScalarFunction::Concat => Self::Concat, BuiltinScalarFunction::ConcatWithSeparator => Self::ConcatWithSeparator, BuiltinScalarFunction::EndsWith => Self::EndsWith, diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 4bf0906685ca..501c51c4be8a 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -282,17 +282,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Expr::ScalarFunction(ScalarFunction::new_udf(fun, args))) } - pub(super) fn sql_named_function_to_expr( - &self, - expr: SQLExpr, - fun: BuiltinScalarFunction, - schema: &DFSchema, - planner_context: &mut PlannerContext, - ) -> Result { - let args = vec![self.sql_expr_to_logical_expr(expr, schema, planner_context)?]; - Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))) - } - pub(super) fn find_window_func( &self, name: &str, diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 7763fa2d8dab..f07377ce50e1 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -28,9 +28,8 @@ use datafusion_expr::expr::AggregateFunctionDefinition; use datafusion_expr::expr::InList; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ - col, expr, lit, AggregateFunction, Between, BinaryExpr, BuiltinScalarFunction, Cast, - Expr, ExprSchemable, GetFieldAccess, GetIndexedField, Like, Literal, Operator, - TryCast, + col, expr, lit, AggregateFunction, Between, BinaryExpr, Cast, Expr, ExprSchemable, + GetFieldAccess, GetIndexedField, Like, Literal, Operator, TryCast, }; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; @@ -522,12 +521,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Ceil { expr, field: _field, - } => self.sql_named_function_to_expr( - *expr, - BuiltinScalarFunction::Ceil, - schema, - planner_context, - ), + } => self.sql_fn_name_to_expr(*expr, "ceil", schema, planner_context), SQLExpr::Overlay { expr, overlay_what, From 34eda15b73a9e278af8844b30ed2f1c21c10359c Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Mon, 15 Apr 2024 19:05:01 +0800 Subject: [PATCH 37/39] feat: support `unnest` multiple arrays (#10044) * Impl find_longest_length * impl unnesting multi columns * Change plans * remove println * fix tests * simplify unnested fields * update doc and tests * more tests * add test * fix comment * Add test for untyped null --------- Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 1 + datafusion/core/src/physical_planner.rs | 9 +- datafusion/expr/src/expr.rs | 20 +- datafusion/expr/src/expr_rewriter/mod.rs | 6 +- datafusion/expr/src/expr_schema.rs | 8 +- datafusion/expr/src/logical_plan/builder.rs | 77 ++- datafusion/expr/src/logical_plan/display.rs | 4 +- datafusion/expr/src/logical_plan/plan.rs | 56 +- datafusion/expr/src/logical_plan/tree_node.rs | 27 +- datafusion/expr/src/tree_node.rs | 2 +- datafusion/physical-plan/Cargo.toml | 1 + datafusion/physical-plan/src/unnest.rs | 651 ++++++++++-------- .../proto/src/logical_plan/from_proto.rs | 7 +- datafusion/proto/src/logical_plan/to_proto.rs | 4 +- .../tests/cases/roundtrip_logical_plan.rs | 2 +- datafusion/sql/src/expr/function.rs | 22 +- datafusion/sql/src/relation/mod.rs | 21 +- datafusion/sql/src/select.rs | 11 +- datafusion/sqllogictest/test_files/unnest.slt | 90 ++- 19 files changed, 577 insertions(+), 442 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 9e192b0be0f1..7ddc0af4306c 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1373,6 +1373,7 @@ dependencies = [ "arrow", "arrow-array", "arrow-buffer", + "arrow-ord", "arrow-schema", "async-trait", "chrono", diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c25523c5ae33..f5e937bb56a0 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1168,12 +1168,13 @@ impl DefaultPhysicalPlanner { Ok(Arc::new(GlobalLimitExec::new(input, *skip, *fetch))) } - LogicalPlan::Unnest(Unnest { input, column, schema, options }) => { + LogicalPlan::Unnest(Unnest { input, columns, schema, options }) => { let input = self.create_initial_plan(input, session_state).await?; - let column_exec = schema.index_of_column(column) - .map(|idx| Column::new(&column.name, idx))?; + let column_execs = columns.iter().map(|column| { + schema.index_of_column(column).map(|idx| Column::new(&column.name, idx)) + }).collect::>()?; let schema = SchemaRef::new(schema.as_ref().to_owned().into()); - Ok(Arc::new(UnnestExec::new(input, column_exec, schema, options.clone()))) + Ok(Arc::new(UnnestExec::new(input, column_execs, schema, options.clone()))) } LogicalPlan::Ddl(ddl) => { // There is no default plan for DDl statements -- diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index c7c50d871902..cffb58dadd8e 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -186,7 +186,16 @@ pub enum Expr { #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct Unnest { - pub exprs: Vec, + pub expr: Box, +} + +impl Unnest { + /// Create a new Unnest expression. + pub fn new(expr: Expr) -> Self { + Self { + expr: Box::new(expr), + } + } } /// Alias expression @@ -1567,8 +1576,8 @@ impl fmt::Display for Expr { } }, Expr::Placeholder(Placeholder { id, .. }) => write!(f, "{id}"), - Expr::Unnest(Unnest { exprs }) => { - write!(f, "UNNEST({exprs:?})") + Expr::Unnest(Unnest { expr }) => { + write!(f, "UNNEST({expr:?})") } } } @@ -1757,7 +1766,10 @@ fn create_name(e: &Expr) -> Result { } } } - Expr::Unnest(Unnest { exprs }) => create_function_name("unnest", false, exprs), + Expr::Unnest(Unnest { expr }) => { + let expr_name = create_name(expr)?; + Ok(format!("unnest({expr_name})")) + } Expr::ScalarFunction(fun) => create_function_name(fun.name(), false, &fun.args), Expr::WindowFunction(WindowFunction { fun, diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index d678fe7ee39c..c11619fc0ea2 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -82,13 +82,13 @@ pub fn normalize_col_with_schemas_and_ambiguity_check( using_columns: &[HashSet], ) -> Result { // Normalize column inside Unnest - if let Expr::Unnest(Unnest { exprs }) = expr { + if let Expr::Unnest(Unnest { expr }) = expr { let e = normalize_col_with_schemas_and_ambiguity_check( - exprs[0].clone(), + expr.as_ref().clone(), schemas, using_columns, )?; - return Ok(Expr::Unnest(Unnest { exprs: vec![e] })); + return Ok(Expr::Unnest(Unnest { expr: Box::new(e) })); } expr.transform(&|expr| { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 39892d9e0c0d..466fd13ce207 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -115,12 +115,8 @@ impl ExprSchemable for Expr { Expr::Case(case) => case.when_then_expr[0].1.get_type(schema), Expr::Cast(Cast { data_type, .. }) | Expr::TryCast(TryCast { data_type, .. }) => Ok(data_type.clone()), - Expr::Unnest(Unnest { exprs }) => { - let arg_data_types = exprs - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - let arg_data_type = arg_data_types[0].clone(); + Expr::Unnest(Unnest { expr }) => { + let arg_data_type = expr.get_type(schema)?; // Unnest's output type is the inner type of the list match arg_data_type{ DataType::List(field) | DataType::LargeList(field) | DataType::FixedSizeList(field, _) =>{ diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index beac5a7f4eb7..f7c0fbac537b 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1112,7 +1112,7 @@ impl LogicalPlanBuilder { /// Unnest the given column. pub fn unnest_column(self, column: impl Into) -> Result { - Ok(Self::from(unnest(self.plan, column.into())?)) + Ok(Self::from(unnest(self.plan, vec![column.into()])?)) } /// Unnest the given column given [`UnnestOptions`] @@ -1123,10 +1123,21 @@ impl LogicalPlanBuilder { ) -> Result { Ok(Self::from(unnest_with_options( self.plan, - column.into(), + vec![column.into()], options, )?)) } + + /// Unnest the given columns with the given [`UnnestOptions`] + pub fn unnest_columns_with_options( + self, + columns: Vec, + options: UnnestOptions, + ) -> Result { + Ok(Self::from(unnest_with_options( + self.plan, columns, options, + )?)) + } } pub fn change_redundant_column(fields: &Fields) -> Vec { let mut name_map = HashMap::new(); @@ -1534,44 +1545,50 @@ impl TableSource for LogicalTableSource { } /// Create a [`LogicalPlan::Unnest`] plan -pub fn unnest(input: LogicalPlan, column: Column) -> Result { - unnest_with_options(input, column, UnnestOptions::new()) +pub fn unnest(input: LogicalPlan, columns: Vec) -> Result { + unnest_with_options(input, columns, UnnestOptions::new()) } /// Create a [`LogicalPlan::Unnest`] plan with options pub fn unnest_with_options( input: LogicalPlan, - column: Column, + columns: Vec, options: UnnestOptions, ) -> Result { - let (unnest_qualifier, unnest_field) = - input.schema().qualified_field_from_column(&column)?; - // Extract the type of the nested field in the list. - let unnested_field = match unnest_field.data_type() { - DataType::List(field) - | DataType::FixedSizeList(field, _) - | DataType::LargeList(field) => Arc::new(Field::new( - unnest_field.name(), - field.data_type().clone(), - unnest_field.is_nullable(), - )), - _ => { - // If the unnest field is not a list type return the input plan. - return Ok(input); - } - }; + let mut unnested_fields: HashMap = HashMap::with_capacity(columns.len()); + // Add qualifiers to the columns. + let mut qualified_columns = Vec::with_capacity(columns.len()); + for c in &columns { + let index = input.schema().index_of_column(c)?; + let (unnest_qualifier, unnest_field) = input.schema().qualified_field(index); + let unnested_field = match unnest_field.data_type() { + DataType::List(field) + | DataType::FixedSizeList(field, _) + | DataType::LargeList(field) => Arc::new(Field::new( + unnest_field.name(), + field.data_type().clone(), + // Unnesting may produce NULLs even if the list is not null. + // For example: unnset([1], []) -> 1, null + true, + )), + _ => { + // If the unnest field is not a list type return the input plan. + return Ok(input); + } + }; + qualified_columns.push(Column::from((unnest_qualifier, unnested_field.as_ref()))); + unnested_fields.insert(index, unnested_field); + } - // Update the schema with the unnest column type changed to contain the nested type. + // Update the schema with the unnest column types changed to contain the nested types. let input_schema = input.schema(); let fields = input_schema .iter() - .map(|(q, f)| { - if f.as_ref() == unnest_field && q == unnest_qualifier { - (unnest_qualifier.cloned(), unnested_field.clone()) - } else { - (q.cloned(), f.clone()) - } + .enumerate() + .map(|(index, (q, f))| match unnested_fields.get(&index) { + Some(unnested_field) => (q.cloned(), unnested_field.clone()), + None => (q.cloned(), f.clone()), }) .collect::>(); @@ -1580,11 +1597,9 @@ pub fn unnest_with_options( // We can use the existing functional dependencies: let deps = input_schema.functional_dependencies().clone(); let schema = Arc::new(df_schema.with_functional_dependencies(deps)?); - let column = Column::from((unnest_qualifier, unnested_field.as_ref())); - Ok(LogicalPlan::Unnest(Unnest { input: Arc::new(input), - column, + columns: qualified_columns, schema, options, })) diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index edc3afd55d63..3a2ed9ffc2d8 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -638,10 +638,10 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { "Node Type": "DescribeTable" }) } - LogicalPlan::Unnest(Unnest { column, .. }) => { + LogicalPlan::Unnest(Unnest { columns, .. }) => { json!({ "Node Type": "Unnest", - "Column": format!("{}", column) + "Column": expr_vec_fmt!(columns), }) } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 91c8670f3805..dbff5046013b 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use super::dml::CopyTo; use super::DdlStatement; -use crate::builder::change_redundant_column; +use crate::builder::{change_redundant_column, unnest_with_options}; use crate::expr::{Alias, Placeholder, Sort as SortExpr, WindowFunction}; use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; @@ -807,51 +807,11 @@ impl LogicalPlan { } LogicalPlan::DescribeTable(_) => Ok(self.clone()), LogicalPlan::Unnest(Unnest { - column, - schema, - options, - .. + columns, options, .. }) => { // Update schema with unnested column type. - let input = Arc::new(inputs.swap_remove(0)); - let (nested_qualifier, nested_field) = - input.schema().qualified_field_from_column(column)?; - let (unnested_qualifier, unnested_field) = - schema.qualified_field_from_column(column)?; - let qualifiers_and_fields = input - .schema() - .iter() - .map(|(qualifier, field)| { - if qualifier.eq(&nested_qualifier) - && field.as_ref() == nested_field - { - ( - unnested_qualifier.cloned(), - Arc::new(unnested_field.clone()), - ) - } else { - (qualifier.cloned(), field.clone()) - } - }) - .collect::>(); - - let schema = Arc::new( - DFSchema::new_with_metadata( - qualifiers_and_fields, - input.schema().metadata().clone(), - )? - // We can use the existing functional dependencies as is: - .with_functional_dependencies( - input.schema().functional_dependencies().clone(), - )?, - ); - - Ok(LogicalPlan::Unnest(Unnest { - input, - column: column.clone(), - schema, - options: options.clone(), - })) + let input = inputs.swap_remove(0); + unnest_with_options(input, columns.clone(), options.clone()) } } } @@ -1581,8 +1541,8 @@ impl LogicalPlan { LogicalPlan::DescribeTable(DescribeTable { .. }) => { write!(f, "DescribeTable") } - LogicalPlan::Unnest(Unnest { column, .. }) => { - write!(f, "Unnest: {column}") + LogicalPlan::Unnest(Unnest { columns, .. }) => { + write!(f, "Unnest: {}", expr_vec_fmt!(columns)) } } } @@ -2556,8 +2516,8 @@ pub enum Partitioning { pub struct Unnest { /// The incoming logical plan pub input: Arc, - /// The column to unnest - pub column: Column, + /// The columns to unnest + pub columns: Vec, /// The output schema, containing the unnested field column. pub schema: DFSchemaRef, /// Options diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 3644f89e8b42..48f047c070dd 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -311,13 +311,13 @@ impl TreeNode for LogicalPlan { } LogicalPlan::Unnest(Unnest { input, - column, + columns, schema, options, }) => rewrite_arc(input, f)?.update_data(|input| { LogicalPlan::Unnest(Unnest { input, - column, + columns, schema, options, }) @@ -507,8 +507,12 @@ impl LogicalPlan { LogicalPlan::TableScan(TableScan { filters, .. }) => { filters.iter().apply_until_stop(f) } - LogicalPlan::Unnest(Unnest { column, .. }) => { - f(&Expr::Column(column.clone())) + LogicalPlan::Unnest(Unnest { columns, .. }) => { + let exprs = columns + .iter() + .map(|c| Expr::Column(c.clone())) + .collect::>(); + exprs.iter().apply_until_stop(f) } LogicalPlan::Distinct(Distinct::On(DistinctOn { on_expr, @@ -706,20 +710,6 @@ impl LogicalPlan { fetch, }) }), - LogicalPlan::Unnest(Unnest { - input, - column, - schema, - options, - }) => f(Expr::Column(column))?.map_data(|column| match column { - Expr::Column(column) => Ok(LogicalPlan::Unnest(Unnest { - input, - column, - schema, - options, - })), - _ => internal_err!("Transformation should return Column"), - })?, LogicalPlan::Distinct(Distinct::On(DistinctOn { on_expr, select_expr, @@ -744,6 +734,7 @@ impl LogicalPlan { }), // plans without expressions LogicalPlan::EmptyRelation(_) + | LogicalPlan::Unnest(_) | LogicalPlan::RecursiveQuery(_) | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 85097f6249e1..35fec509c95a 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -36,6 +36,7 @@ impl TreeNode for Expr { ) -> Result { let children = match self { Expr::Alias(Alias{expr,..}) + | Expr::Unnest(Unnest{expr}) | Expr::Not(expr) | Expr::IsNotNull(expr) | Expr::IsTrue(expr) @@ -60,7 +61,6 @@ impl TreeNode for Expr { GetFieldAccess::NamedStructField { .. } => vec![expr], } } - Expr::Unnest(Unnest { exprs }) | Expr::GroupingSet(GroupingSet::Rollup(exprs)) | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.iter().collect(), Expr::ScalarFunction (ScalarFunction{ args, .. } ) => { diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 6a78bd596a46..6863f2646000 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -39,6 +39,7 @@ ahash = { version = "0.8", default-features = false, features = [ arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } +arrow-ord = { workspace = true } arrow-schema = { workspace = true } async-trait = { workspace = true } chrono = { workspace = true } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 6ea1b3c40c83..45b848112ba9 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -15,8 +15,9 @@ // specific language governing permissions and limitations // under the License. -//! Defines the unnest column plan for unnesting values in a column that contains a list -//! type, conceptually is like joining each row with all the values in the list column. +//! Define a plan for unnesting values in columns that contain a list type. + +use std::collections::HashMap; use std::{any::Any, sync::Arc}; use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; @@ -27,15 +28,17 @@ use crate::{ }; use arrow::array::{ - Array, ArrayRef, ArrowPrimitiveType, FixedSizeListArray, GenericListArray, - LargeListArray, ListArray, OffsetSizeTrait, PrimitiveArray, -}; -use arrow::compute::kernels; -use arrow::datatypes::{ - ArrowNativeType, DataType, Int32Type, Int64Type, Schema, SchemaRef, + Array, ArrayRef, AsArray, FixedSizeListArray, LargeListArray, ListArray, + PrimitiveArray, }; +use arrow::compute::kernels::length::length; +use arrow::compute::kernels::zip::zip; +use arrow::compute::{cast, is_not_null, kernels, sum}; +use arrow::datatypes::{DataType, Int64Type, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::{exec_err, Result, UnnestOptions}; +use arrow_array::{Int64Array, Scalar}; +use arrow_ord::cmp::lt; +use datafusion_common::{exec_datafusion_err, exec_err, Result, UnnestOptions}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; @@ -43,7 +46,7 @@ use async_trait::async_trait; use futures::{Stream, StreamExt}; use log::trace; -/// Unnest the given column by joining the row with each value in the +/// Unnest the given columns by joining the row with each value in the /// nested type. /// /// See [`UnnestOptions`] for more details and an example. @@ -53,8 +56,8 @@ pub struct UnnestExec { input: Arc, /// The schema once the unnest is applied schema: SchemaRef, - /// The unnest column - column: Column, + /// The unnest columns + columns: Vec, /// Options options: UnnestOptions, /// Execution metrics @@ -67,7 +70,7 @@ impl UnnestExec { /// Create a new [UnnestExec]. pub fn new( input: Arc, - column: Column, + columns: Vec, schema: SchemaRef, options: UnnestOptions, ) -> Self { @@ -75,7 +78,7 @@ impl UnnestExec { UnnestExec { input, schema, - column, + columns, options, metrics: Default::default(), cache, @@ -134,7 +137,7 @@ impl ExecutionPlan for UnnestExec { ) -> Result> { Ok(Arc::new(UnnestExec::new( children[0].clone(), - self.column.clone(), + self.columns.clone(), self.schema.clone(), self.options.clone(), ))) @@ -155,7 +158,7 @@ impl ExecutionPlan for UnnestExec { Ok(Box::pin(UnnestStream { input, schema: self.schema.clone(), - column: self.column.clone(), + columns: self.columns.clone(), options: self.options.clone(), metrics, })) @@ -210,8 +213,8 @@ struct UnnestStream { input: SendableRecordBatchStream, /// Unnested schema schema: Arc, - /// The unnest column - column: Column, + /// The unnest columns + columns: Vec, /// Options options: UnnestOptions, /// Metrics @@ -249,7 +252,7 @@ impl UnnestStream { Some(Ok(batch)) => { let timer = self.metrics.elapsed_compute.timer(); let result = - build_batch(&batch, &self.schema, &self.column, &self.options); + build_batch(&batch, &self.schema, &self.columns, &self.options); self.metrics.input_batches.add(1); self.metrics.input_rows.add(batch.num_rows()); if let Ok(ref batch) = result { @@ -276,270 +279,265 @@ impl UnnestStream { } } +/// For each row in a `RecordBatch`, some list columns need to be unnested. +/// We will expand the values in each list into multiple rows, +/// taking the longest length among these lists, and shorter lists are padded with NULLs. +// +/// For columns that don't need to be unnested, repeat their values until reaching the longest length. fn build_batch( batch: &RecordBatch, schema: &SchemaRef, - column: &Column, + columns: &[Column], options: &UnnestOptions, ) -> Result { - let list_array = column.evaluate(batch)?.into_array(batch.num_rows())?; - match list_array.data_type() { - DataType::List(_) => { - let list_array = list_array.as_any().downcast_ref::().unwrap(); - build_batch_generic_list::( - batch, - schema, - column.index(), - list_array, - options, - ) - } - DataType::LargeList(_) => { - let list_array = list_array - .as_any() - .downcast_ref::() - .unwrap(); - build_batch_generic_list::( - batch, - schema, - column.index(), - list_array, - options, - ) - } - DataType::FixedSizeList(_, _) => { - let list_array = list_array - .as_any() - .downcast_ref::() - .unwrap(); - build_batch_fixedsize_list(batch, schema, column.index(), list_array, options) - } - _ => exec_err!("Invalid unnest column {column}"), + let list_arrays: Vec = columns + .iter() + .map(|column| column.evaluate(batch)?.into_array(batch.num_rows())) + .collect::>()?; + + let longest_length = find_longest_length(&list_arrays, options)?; + let unnested_length = longest_length.as_primitive::(); + let total_length = if unnested_length.is_empty() { + 0 + } else { + sum(unnested_length).ok_or_else(|| { + exec_datafusion_err!("Failed to calculate the total unnested length") + })? as usize + }; + if total_length == 0 { + return Ok(RecordBatch::new_empty(schema.clone())); } -} -fn build_batch_generic_list>( - batch: &RecordBatch, - schema: &SchemaRef, - unnest_column_idx: usize, - list_array: &GenericListArray, - options: &UnnestOptions, -) -> Result { - let unnested_array = unnest_generic_list::(list_array, options)?; - - let take_indicies = - create_take_indicies_generic::(list_array, unnested_array.len(), options); - - batch_from_indices( - batch, - schema, - unnest_column_idx, - &unnested_array, - &take_indicies, - ) + // Unnest all the list arrays + let unnested_arrays = + unnest_list_arrays(&list_arrays, unnested_length, total_length)?; + let unnested_array_map: HashMap<_, _> = unnested_arrays + .into_iter() + .zip(columns.iter()) + .map(|(array, column)| (column.index(), array)) + .collect(); + + // Create the take indices array for other columns + let take_indicies = create_take_indicies(unnested_length, total_length); + + batch_from_indices(batch, schema, &unnested_array_map, &take_indicies) } -/// Given this `GenericList` list_array: +/// Find the longest list length among the given list arrays for each row. +/// +/// For example if we have the following two list arrays: /// /// ```ignore -/// [1], null, [2, 3, 4], null, [5, 6] +/// l1: [1, 2, 3], null, [], [3] +/// l2: [4,5], [], null, [6, 7] /// ``` -/// Its values array is represented like this: +/// +/// If `preserve_nulls` is false, the longest length array will be: /// /// ```ignore -/// [1, 2, 3, 4, 5, 6] +/// longest_length: [3, 0, 0, 2] /// ``` /// -/// So if there are no null values or `UnnestOptions.preserve_nulls` is false -/// we can return the values array without any copying. +/// whereas if `preserve_nulls` is true, the longest length array will be: /// -/// Otherwise we'll transfrom the values array using the take kernel and the following take indicies: /// /// ```ignore -/// 0, null, 1, 2, 3, null, 4, 5 +/// longest_length: [3, 1, 1, 2] /// ``` /// -fn unnest_generic_list>( - list_array: &GenericListArray, +fn find_longest_length( + list_arrays: &[ArrayRef], options: &UnnestOptions, -) -> Result> { - let values = list_array.values(); - if list_array.null_count() == 0 { - return Ok(values.clone()); +) -> Result { + // The length of a NULL list + let null_length = if options.preserve_nulls { + Scalar::new(Int64Array::from_value(1, 1)) + } else { + Scalar::new(Int64Array::from_value(0, 1)) + }; + let list_lengths: Vec = list_arrays + .iter() + .map(|list_array| { + let mut length_array = length(list_array)?; + // Make sure length arrays have the same type. Int64 is the most general one. + length_array = cast(&length_array, &DataType::Int64)?; + length_array = + zip(&is_not_null(&length_array)?, &length_array, &null_length)?; + Ok(length_array) + }) + .collect::>()?; + + let longest_length = list_lengths.iter().skip(1).try_fold( + list_lengths[0].clone(), + |longest, current| { + let is_lt = lt(&longest, ¤t)?; + zip(&is_lt, ¤t, &longest) + }, + )?; + Ok(longest_length) +} + +/// Trait defining common methods used for unnesting, implemented by list array types. +trait ListArrayType: Array { + /// Returns a reference to the values of this list. + fn values(&self) -> &ArrayRef; + + /// Returns the start and end offset of the values for the given row. + fn value_offsets(&self, row: usize) -> (i64, i64); +} + +impl ListArrayType for ListArray { + fn values(&self) -> &ArrayRef { + self.values() } - let mut take_indicies_builder = - PrimitiveArray::

::builder(values.len() + list_array.null_count()); - let offsets = list_array.value_offsets(); - for row in 0..list_array.len() { - if list_array.is_null(row) { - if options.preserve_nulls { - take_indicies_builder.append_null(); - } - } else { - let start = offsets[row].as_usize(); - let end = offsets[row + 1].as_usize(); - for idx in start..end { - take_indicies_builder.append_value(P::Native::from_usize(idx).unwrap()); - } - } + fn value_offsets(&self, row: usize) -> (i64, i64) { + let offsets = self.value_offsets(); + (offsets[row].into(), offsets[row + 1].into()) } - Ok(kernels::take::take( - &values, - &take_indicies_builder.finish(), - None, - )?) } -fn build_batch_fixedsize_list( - batch: &RecordBatch, - schema: &SchemaRef, - unnest_column_idx: usize, - list_array: &FixedSizeListArray, - options: &UnnestOptions, -) -> Result { - let unnested_array = unnest_fixed_list(list_array, options)?; - - let take_indicies = - create_take_indicies_fixed(list_array, unnested_array.len(), options); - - batch_from_indices( - batch, - schema, - unnest_column_idx, - &unnested_array, - &take_indicies, - ) +impl ListArrayType for LargeListArray { + fn values(&self) -> &ArrayRef { + self.values() + } + + fn value_offsets(&self, row: usize) -> (i64, i64) { + let offsets = self.value_offsets(); + (offsets[row], offsets[row + 1]) + } } -/// Given this `FixedSizeListArray` list_array: +impl ListArrayType for FixedSizeListArray { + fn values(&self) -> &ArrayRef { + self.values() + } + + fn value_offsets(&self, row: usize) -> (i64, i64) { + let start = self.value_offset(row) as i64; + (start, start + self.value_length() as i64) + } +} + +/// Unnest multiple list arrays according to the length array. +fn unnest_list_arrays( + list_arrays: &[ArrayRef], + length_array: &PrimitiveArray, + capacity: usize, +) -> Result> { + let typed_arrays = list_arrays + .iter() + .map(|list_array| match list_array.data_type() { + DataType::List(_) => Ok(list_array.as_list::() as &dyn ListArrayType), + DataType::LargeList(_) => { + Ok(list_array.as_list::() as &dyn ListArrayType) + } + DataType::FixedSizeList(_, _) => { + Ok(list_array.as_fixed_size_list() as &dyn ListArrayType) + } + other => exec_err!("Invalid unnest datatype {other }"), + }) + .collect::>>()?; + + // If there is only one list column to unnest and it doesn't contain any NULL lists, + // we can return the values array directly without any copying. + if typed_arrays.len() == 1 && typed_arrays[0].null_count() == 0 { + Ok(vec![typed_arrays[0].values().clone()]) + } else { + typed_arrays + .iter() + .map(|list_array| unnest_list_array(*list_array, length_array, capacity)) + .collect::>() + } +} + +/// Unnest a list array according the target length array. /// -/// ```ignore -/// [1, 2], null, [3, 4], null, [5, 6] -/// ``` -/// Its values array is represented like this: +/// Consider a list array like this: /// /// ```ignore -/// [1, 2, null, null 3, 4, null, null, 5, 6] +/// [1], [2, 3, 4], null, [5], [], /// ``` /// -/// So if there are no null values -/// we can return the values array without any copying. -/// -/// Otherwise we'll transfrom the values array using the take kernel. -/// -/// If `UnnestOptions.preserve_nulls` is true the take indicies will look like this: +/// and the length array is: /// /// ```ignore -/// 0, 1, null, 4, 5, null, 8, 9 +/// [2, 3, 2, 1, 2] /// ``` -/// Otherwise we drop the nulls and take indicies will look like this: +/// +/// If the length of a certain list is less than the target length, pad with NULLs. +/// So the unnested array will look like this: /// /// ```ignore -/// 0, 1, 4, 5, 8, 9 +/// [1, null, 2, 3, 4, null, null, 5, null, null] /// ``` /// -fn unnest_fixed_list( - list_array: &FixedSizeListArray, - options: &UnnestOptions, -) -> Result> { +fn unnest_list_array( + list_array: &dyn ListArrayType, + length_array: &PrimitiveArray, + capacity: usize, +) -> Result { let values = list_array.values(); - - if list_array.null_count() == 0 { - Ok(values.clone()) - } else { - let len_without_nulls = - values.len() - list_array.null_count() * list_array.value_length() as usize; - let null_count = if options.preserve_nulls { - list_array.null_count() - } else { - 0 - }; - let mut builder = - PrimitiveArray::::builder(len_without_nulls + null_count); - let mut take_offset = 0; - let fixed_value_length = list_array.value_length() as usize; - list_array.iter().for_each(|elem| match elem { - Some(_) => { - for i in 0..fixed_value_length { - //take_offset + i is always positive - let take_index = take_offset + i; - builder.append_value(take_index as i32); - } - take_offset += fixed_value_length; - } - None => { - if options.preserve_nulls { - builder.append_null(); - } - take_offset += fixed_value_length; + let mut take_indicies_builder = PrimitiveArray::::builder(capacity); + for row in 0..list_array.len() { + let mut value_length = 0; + if !list_array.is_null(row) { + let (start, end) = list_array.value_offsets(row); + value_length = end - start; + for i in start..end { + take_indicies_builder.append_value(i) } - }); - Ok(kernels::take::take(&values, &builder.finish(), None)?) + } + let target_length = length_array.value(row); + debug_assert!( + value_length <= target_length, + "value length is beyond the longest length" + ); + // Pad with NULL values + for _ in value_length..target_length { + take_indicies_builder.append_null(); + } } + Ok(kernels::take::take( + &values, + &take_indicies_builder.finish(), + None, + )?) } -/// Creates take indicies to be used to expand all other column's data. -/// Every column value needs to be repeated as many times as many elements there is in each corresponding array value. +/// Creates take indicies that will be used to expand all columns except for the unnest [`columns`](UnnestExec::columns). +/// Every column value needs to be repeated multiple times according to the length array. /// -/// If the column being unnested looks like this: +/// If the length array looks like this: /// /// ```ignore -/// [1], null, [2, 3, 4], null, [5, 6] +/// [2, 3, 1] /// ``` -/// Then `create_take_indicies_generic` will return an array like this +/// Then `create_take_indicies` will return an array like this /// /// ```ignore -/// [1, null, 2, 2, 2, null, 4, 4] +/// [0, 0, 1, 1, 1, 2] /// ``` /// -fn create_take_indicies_generic>( - list_array: &GenericListArray, +fn create_take_indicies( + length_array: &PrimitiveArray, capacity: usize, - options: &UnnestOptions, -) -> PrimitiveArray

{ - let mut builder = PrimitiveArray::

::builder(capacity); - let null_repeat: usize = if options.preserve_nulls { 1 } else { 0 }; - - for row in 0..list_array.len() { - let repeat = if list_array.is_null(row) { - null_repeat - } else { - list_array.value(row).len() - }; - - // `index` is a positive interger. - let index = P::Native::from_usize(row).unwrap(); - (0..repeat).for_each(|_| builder.append_value(index)); +) -> PrimitiveArray { + // `find_longest_length()` guarantees this. + debug_assert!( + length_array.null_count() == 0, + "length array should not contain nulls" + ); + let mut builder = PrimitiveArray::::builder(capacity); + for (index, repeat) in length_array.iter().enumerate() { + // The length array should not contain nulls, so unwrap is safe + let repeat = repeat.unwrap(); + (0..repeat).for_each(|_| builder.append_value(index as i64)); } - builder.finish() } -fn create_take_indicies_fixed( - list_array: &FixedSizeListArray, - capacity: usize, - options: &UnnestOptions, -) -> PrimitiveArray { - let mut builder = PrimitiveArray::::builder(capacity); - let null_repeat: usize = if options.preserve_nulls { 1 } else { 0 }; - - for row in 0..list_array.len() { - let repeat = if list_array.is_null(row) { - null_repeat - } else { - list_array.value_length() as usize - }; - - // `index` is a positive interger. - let index = ::Native::from_usize(row).unwrap(); - (0..repeat).for_each(|_| builder.append_value(index)); - } - - builder.finish() -} - -/// Create the final batch given the unnested column array and a `indices` array +/// Create the final batch given the unnested column arrays and a `indices` array /// that is used by the take kernel to copy values. /// /// For example if we have the following `RecordBatch`: @@ -549,8 +547,8 @@ fn create_take_indicies_fixed( /// c2: 'a', 'b', 'c', null, 'd' /// ``` /// -/// then the `unnested_array` contains the unnest column that will replace `c1` in -/// the final batch: +/// then the `unnested_list_arrays` contains the unnest column that will replace `c1` in +/// the final batch if `preserve_nulls` is true: /// /// ```ignore /// c1: 1, null, 2, 3, 4, null, 5, 6 @@ -570,26 +568,19 @@ fn create_take_indicies_fixed( /// c2: 'a', 'b', 'c', 'c', 'c', null, 'd', 'd' /// ``` /// -fn batch_from_indices( +fn batch_from_indices( batch: &RecordBatch, schema: &SchemaRef, - unnest_column_idx: usize, - unnested_array: &ArrayRef, - indices: &PrimitiveArray, -) -> Result -where - T: ArrowPrimitiveType, -{ + unnested_list_arrays: &HashMap, + indices: &PrimitiveArray, +) -> Result { let arrays = batch .columns() .iter() .enumerate() - .map(|(col_idx, arr)| { - if col_idx == unnest_column_idx { - Ok(unnested_array.clone()) - } else { - Ok(kernels::take::take(&arr, indices, None)?) - } + .map(|(col_idx, arr)| match unnested_list_arrays.get(&col_idx) { + Some(unnested_array) => Ok(unnested_array.clone()), + None => Ok(kernels::take::take(arr, indices, None)?), }) .collect::>>()?; @@ -599,51 +590,51 @@ where #[cfg(test)] mod tests { use super::*; - use arrow::{ - array::AsArray, - datatypes::{DataType, Field}, - }; - use arrow_array::StringArray; + use arrow::datatypes::{DataType, Field}; + use arrow_array::{GenericListArray, OffsetSizeTrait, StringArray}; use arrow_buffer::{BooleanBufferBuilder, NullBuffer, OffsetBuffer}; - // Create a ListArray with the following list values: + // Create a GenericListArray with the following list values: // [A, B, C], [], NULL, [D], NULL, [NULL, F] - fn make_test_array() -> ListArray { + fn make_generic_array() -> GenericListArray + where + OffsetSize: OffsetSizeTrait, + { let mut values = vec![]; - let mut offsets = vec![0]; - let mut valid = BooleanBufferBuilder::new(2); + let mut offsets: Vec = vec![OffsetSize::zero()]; + let mut valid = BooleanBufferBuilder::new(6); // [A, B, C] values.extend_from_slice(&[Some("A"), Some("B"), Some("C")]); - offsets.push(values.len() as i32); + offsets.push(OffsetSize::from_usize(values.len()).unwrap()); valid.append(true); // [] - offsets.push(values.len() as i32); + offsets.push(OffsetSize::from_usize(values.len()).unwrap()); valid.append(true); // NULL with non-zero value length // Issue https://github.com/apache/arrow-datafusion/issues/9932 values.push(Some("?")); - offsets.push(values.len() as i32); + offsets.push(OffsetSize::from_usize(values.len()).unwrap()); valid.append(false); // [D] values.push(Some("D")); - offsets.push(values.len() as i32); + offsets.push(OffsetSize::from_usize(values.len()).unwrap()); valid.append(true); // Another NULL with zero value length - offsets.push(values.len() as i32); + offsets.push(OffsetSize::from_usize(values.len()).unwrap()); valid.append(false); // [NULL, F] values.extend_from_slice(&[None, Some("F")]); - offsets.push(values.len() as i32); + offsets.push(OffsetSize::from_usize(values.len()).unwrap()); valid.append(true); let field = Arc::new(Field::new("item", DataType::Utf8, true)); - ListArray::new( + GenericListArray::::new( field, OffsetBuffer::new(offsets.into()), Arc::new(StringArray::from(values)), @@ -651,43 +642,141 @@ mod tests { ) } - #[test] - fn test_unnest_generic_list() -> datafusion_common::Result<()> { - let list_array = make_test_array(); - - // Test with preserve_nulls = false - let options = UnnestOptions { - preserve_nulls: false, - }; - let unnested_array = - unnest_generic_list::(&list_array, &options)?; - let strs = unnested_array.as_string::().iter().collect::>(); - assert_eq!( - strs, - vec![Some("A"), Some("B"), Some("C"), Some("D"), None, Some("F")] - ); + // Create a FixedSizeListArray with the following list values: + // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] + fn make_fixed_list() -> FixedSizeListArray { + let values = Arc::new(StringArray::from_iter([ + Some("A"), + Some("B"), + None, + None, + Some("C"), + Some("D"), + None, + None, + None, + Some("F"), + None, + None, + ])); + let field = Arc::new(Field::new("item", DataType::Utf8, true)); + let valid = NullBuffer::from(vec![true, false, true, false, true, true]); + FixedSizeListArray::new(field, 2, values, Some(valid)) + } - // Test with preserve_nulls = true - let options = UnnestOptions { - preserve_nulls: true, - }; - let unnested_array = - unnest_generic_list::(&list_array, &options)?; + fn verify_unnest_list_array( + list_array: &dyn ListArrayType, + lengths: Vec, + expected: Vec>, + ) -> datafusion_common::Result<()> { + let length_array = Int64Array::from(lengths); + let unnested_array = unnest_list_array(list_array, &length_array, 3 * 6)?; let strs = unnested_array.as_string::().iter().collect::>(); - assert_eq!( - strs, + assert_eq!(strs, expected); + Ok(()) + } + + #[test] + fn test_unnest_list_array() -> datafusion_common::Result<()> { + // [A, B, C], [], NULL, [D], NULL, [NULL, F] + let list_array = make_generic_array::(); + verify_unnest_list_array( + &list_array, + vec![3, 2, 1, 2, 0, 3], vec![ Some("A"), Some("B"), Some("C"), None, + None, + None, Some("D"), None, None, - Some("F") - ] + Some("F"), + None, + ], + )?; + + // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] + let list_array = make_fixed_list(); + verify_unnest_list_array( + &list_array, + vec![3, 1, 2, 0, 2, 3], + vec![ + Some("A"), + Some("B"), + None, + None, + Some("C"), + Some("D"), + None, + Some("F"), + None, + None, + None, + ], + )?; + + Ok(()) + } + + fn verify_longest_length( + list_arrays: &[ArrayRef], + preserve_nulls: bool, + expected: Vec, + ) -> datafusion_common::Result<()> { + let options = UnnestOptions { preserve_nulls }; + let longest_length = find_longest_length(list_arrays, &options)?; + let expected_array = Int64Array::from(expected); + assert_eq!( + longest_length + .as_any() + .downcast_ref::() + .unwrap(), + &expected_array ); + Ok(()) + } + + #[test] + fn test_longest_list_length() -> datafusion_common::Result<()> { + // Test with single ListArray + // [A, B, C], [], NULL, [D], NULL, [NULL, F] + let list_array = Arc::new(make_generic_array::()) as ArrayRef; + verify_longest_length(&[list_array.clone()], false, vec![3, 0, 0, 1, 0, 2])?; + verify_longest_length(&[list_array.clone()], true, vec![3, 0, 1, 1, 1, 2])?; + + // Test with single LargeListArray + // [A, B, C], [], NULL, [D], NULL, [NULL, F] + let list_array = Arc::new(make_generic_array::()) as ArrayRef; + verify_longest_length(&[list_array.clone()], false, vec![3, 0, 0, 1, 0, 2])?; + verify_longest_length(&[list_array.clone()], true, vec![3, 0, 1, 1, 1, 2])?; + + // Test with single FixedSizeListArray + // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] + let list_array = Arc::new(make_fixed_list()) as ArrayRef; + verify_longest_length(&[list_array.clone()], false, vec![2, 0, 2, 0, 2, 2])?; + verify_longest_length(&[list_array.clone()], true, vec![2, 1, 2, 1, 2, 2])?; + + // Test with multiple list arrays + // [A, B, C], [], NULL, [D], NULL, [NULL, F] + // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] + let list1 = Arc::new(make_generic_array::()) as ArrayRef; + let list2 = Arc::new(make_fixed_list()) as ArrayRef; + let list_arrays = vec![list1.clone(), list2.clone()]; + verify_longest_length(&list_arrays, false, vec![3, 0, 2, 1, 2, 2])?; + verify_longest_length(&list_arrays, true, vec![3, 1, 2, 1, 2, 2])?; Ok(()) } + + #[test] + fn test_create_take_indicies() -> datafusion_common::Result<()> { + let length_array = Int64Array::from(vec![2, 3, 1]); + let take_indicies = create_take_indicies(&length_array, 6); + let expected = Int64Array::from(vec![0, 0, 1, 1, 1, 2]); + assert_eq!(take_indicies, expected); + Ok(()) + } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 4e61385bb545..e66bd1a5f0a9 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1257,8 +1257,11 @@ pub fn parse_expr( parse_required_expr(negative.expr.as_deref(), registry, "expr", codec)?, ))), ExprType::Unnest(unnest) => { - let exprs = parse_exprs(&unnest.exprs, registry, codec)?; - Ok(Expr::Unnest(Unnest { exprs })) + let mut exprs = parse_exprs(&unnest.exprs, registry, codec)?; + if exprs.len() != 1 { + return Err(proto_error("Unnest must have exactly one expression")); + } + Ok(Expr::Unnest(Unnest::new(exprs.swap_remove(0)))) } ExprType::InList(in_list) => Ok(Expr::InList(InList::new( Box::new(parse_required_expr( diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 2680bc15e1b4..4916b4bed9a3 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -963,9 +963,9 @@ pub fn serialize_expr( expr_type: Some(ExprType::Negative(expr)), } } - Expr::Unnest(Unnest { exprs }) => { + Expr::Unnest(Unnest { expr }) => { let expr = protobuf::Unnest { - exprs: serialize_exprs(exprs, codec)?, + exprs: vec![serialize_expr(expr.as_ref(), codec)?], }; protobuf::LogicalExprNode { expr_type: Some(ExprType::Unnest(expr)), diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index e680a1b2ff1e..eee15008fbbb 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -1599,7 +1599,7 @@ fn roundtrip_inlist() { #[test] fn roundtrip_unnest() { let test_expr = Expr::Unnest(Unnest { - exprs: vec![lit(1), lit(2), lit(3)], + expr: Box::new(col("col")), }); let ctx = SessionContext::new(); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 501c51c4be8a..c225afec58d6 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -119,10 +119,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Build Unnest expression if name.eq("unnest") { - let exprs = + let mut exprs = self.function_args_to_expr(args.clone(), schema, planner_context)?; - Self::check_unnest_args(&exprs, schema)?; - return Ok(Expr::Unnest(Unnest { exprs })); + if exprs.len() != 1 { + return plan_err!("unnest() requires exactly one argument"); + } + let expr = exprs.swap_remove(0); + Self::check_unnest_arg(&expr, schema)?; + return Ok(Expr::Unnest(Unnest::new(expr))); } // next, scalar built-in @@ -342,17 +346,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .collect::>>() } - pub(crate) fn check_unnest_args(args: &[Expr], schema: &DFSchema) -> Result<()> { - // Currently only one argument is supported - let arg = match args.len() { - 0 => { - return plan_err!("unnest() requires at least one argument"); - } - 1 => &args[0], - _ => { - return not_impl_err!("unnest() does not support multiple arguments yet"); - } - }; + pub(crate) fn check_unnest_arg(arg: &Expr, schema: &DFSchema) -> Result<()> { // Check argument type, array types are supported match arg.get_type(schema)? { DataType::List(_) diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index 1e01205ba618..9380e569f2e4 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -105,15 +105,24 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Unnest table factor has empty input let schema = DFSchema::empty(); let input = LogicalPlanBuilder::empty(true).build()?; - let exprs = array_exprs + // Unnest table factor can have multiple arugments. + // We treat each argument as a separate unnest expression. + let unnest_exprs = array_exprs .into_iter() - .map(|expr| { - self.sql_expr_to_logical_expr(expr, &schema, planner_context) + .map(|sql_expr| { + let expr = self.sql_expr_to_logical_expr( + sql_expr, + &schema, + planner_context, + )?; + Self::check_unnest_arg(&expr, &schema)?; + Ok(Expr::Unnest(Unnest::new(expr))) }) .collect::>>()?; - Self::check_unnest_args(&exprs, &schema)?; - let unnest_expr = Expr::Unnest(Unnest { exprs }); - let logical_plan = self.try_process_unnest(input, vec![unnest_expr])?; + if unnest_exprs.is_empty() { + return plan_err!("UNNEST must have at least one argument"); + } + let logical_plan = self.try_process_unnest(input, unnest_exprs)?; (logical_plan, alias) } TableFactor::UNNEST { .. } => { diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 1bfd60a8ce1a..30eacdb44c4a 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -294,13 +294,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { transformed, tnr: _, } = expr.transform_up_mut(&mut |expr: Expr| { - if let Expr::Unnest(Unnest { ref exprs }) = expr { + if let Expr::Unnest(Unnest { expr: ref arg }) = expr { let column_name = expr.display_name()?; unnest_columns.push(column_name.clone()); // Add alias for the argument expression, to avoid naming conflicts with other expressions // in the select list. For example: `select unnest(col1), col1 from t`. inner_projection_exprs - .push(exprs[0].clone().alias(column_name.clone())); + .push(arg.clone().alias(column_name.clone())); Ok(Transformed::yes(Expr::Column(Column::from_name( column_name, )))) @@ -332,15 +332,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .project(inner_projection_exprs)? .build() } else { - if unnest_columns.len() > 1 { - return not_impl_err!("Only support single unnest expression for now"); - } - let unnest_column = unnest_columns.pop().unwrap(); + let columns = unnest_columns.into_iter().map(|col| col.into()).collect(); // Set preserve_nulls to false to ensure compatibility with DuckDB and PostgreSQL let unnest_options = UnnestOptions::new().with_preserve_nulls(false); LogicalPlanBuilder::from(input) .project(inner_projection_exprs)? - .unnest_column_with_options(unnest_column, unnest_options)? + .unnest_columns_with_options(columns, unnest_options)? .project(outer_projection_exprs)? .build() } diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index 5c178bb392b1..38207fa7d1d6 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -22,12 +22,12 @@ statement ok CREATE TABLE unnest_table AS VALUES - ([1,2,3], [7], 1), - ([4,5], [8,9,10], 2), - ([6], [11,12], 3), - ([12], [null, 42, null], null), + ([1,2,3], [7], 1, [13, 14]), + ([4,5], [8,9,10], 2, [15, 16]), + ([6], [11,12], 3, null), + ([12], [null, 42, null], null, null), -- null array to verify the `preserve_nulls` option - (null, null, 4) + (null, null, 4, [17, 18]) ; ## Basic unnest expression in select list @@ -93,6 +93,20 @@ NULL 42 NULL +## Unnest single column and filter out null lists +query I +select unnest(column2) from unnest_table where column2 is not null; +---- +7 +8 +9 +10 +11 +12 +NULL +42 +NULL + ## Unnest with additional column ## Issue: https://github.com/apache/arrow-datafusion/issues/9349 query II @@ -135,9 +149,48 @@ select array_remove(column1, 4), unnest(column2), column3 * 10 from unnest_table query error DataFusion error: Error during planning: unnest\(\) can only be applied to array, struct and null select unnest(column3) from unnest_table; +## Unnest doesn't work with untyped nulls +query error DataFusion error: This feature is not implemented: unnest\(\) does not support null yet +select unnest(null) from unnest_table; + ## Multiple unnest functions in selection -query error DataFusion error: This feature is not implemented: Only support single unnest expression for now -select unnest(column1), unnest(column2) from unnest_table; +query ?I +select unnest([]), unnest(NULL::int[]); +---- + +query III +select + unnest(column1), + unnest(arrow_cast(column2, 'LargeList(Int64)')), + unnest(arrow_cast(column4, 'FixedSizeList(2, Int64)')) +from unnest_table where column4 is not null; +---- +1 7 13 +2 NULL 14 +3 NULL NULL +4 8 15 +5 9 16 +NULL 10 NULL +NULL NULL 17 +NULL NULL 18 + +query IIII +select + unnest(column1), unnest(column2) + 2, + column3 * 10, unnest(array_remove(column1, '4')) +from unnest_table; +---- +1 9 10 1 +2 NULL 10 2 +3 NULL 10 3 +4 10 20 5 +5 11 20 NULL +NULL 12 20 NULL +6 13 30 6 +NULL 14 30 NULL +12 NULL NULL 12 +NULL 44 NULL NULL +NULL NULL NULL NULL ## Unnest scalar in select list query error DataFusion error: Error during planning: unnest\(\) can only be applied to array, struct and null @@ -149,7 +202,7 @@ select * from unnest(1); ## Unnest empty expression in select list -query error DataFusion error: Error during planning: unnest\(\) requires at least one argument +query error DataFusion error: Error during planning: unnest\(\) requires exactly one argument select unnest(); ## Unnest empty expression in from clause @@ -157,13 +210,26 @@ query error DataFusion error: SQL error: ParserError\("Expected an expression:, select * from unnest(); -## Unnest multiple expressions in select list -query error DataFusion error: This feature is not implemented: unnest\(\) does not support multiple arguments yet +## Unnest multiple expressions in select list. This form is only allowed in a query's FROM clause. +query error DataFusion error: Error during planning: unnest\(\) requires exactly one argument select unnest([1,2], [2,3]); ## Unnest multiple expressions in from clause -query error DataFusion error: This feature is not implemented: unnest\(\) does not support multiple arguments yet -select * from unnest([1,2], [2,3]); +query ITII +select * from unnest( + [1,2], + arrow_cast(['a','b', 'c'], 'LargeList(Utf8)'), + arrow_cast([4, NULL], 'FixedSizeList(2, Int64)'), + NULL::int[] +) as t(a, b, c, d); +---- +1 a 4 NULL +2 b NULL NULL +NULL c NULL NULL + +query ?I +select * from unnest([], NULL::int[]); +---- ## Unnest struct expression in select list From 060e67e8ff3b57ab695daeb28cf7175c4e2c568c Mon Sep 17 00:00:00 2001 From: Kaviraj Kanagaraj Date: Mon, 15 Apr 2024 13:40:19 +0200 Subject: [PATCH 38/39] cleanup(tests): Move tests from `push_down_projections.rs` to `optimize_projections.rs` (#10071) * cleanup(tests): Move tests from `push_down_projections.rs` to `optimize_projections.rs` Fixes: #9978 The PR #8340 removed `push_down_projections.rs` in favour of `optimize_projections.rs`. This PR moves the tests as well. Signed-off-by: Kaviraj * remove the file `push_down_projection.rs` Signed-off-by: Kaviraj * Fix method signatures that are broken by other PRs Signed-off-by: Kaviraj * remove `push_down_projections.rs` from `lib.rs` Signed-off-by: Kaviraj --------- Signed-off-by: Kaviraj --- datafusion/optimizer/src/lib.rs | 1 - .../optimizer/src/optimize_projections.rs | 637 ++++++++++++++++- .../optimizer/src/push_down_projection.rs | 660 ------------------ 3 files changed, 631 insertions(+), 667 deletions(-) delete mode 100644 datafusion/optimizer/src/push_down_projection.rs diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index f1f49727c39c..9176d67c1d18 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -47,7 +47,6 @@ pub mod optimizer; pub mod propagate_empty_relation; pub mod push_down_filter; pub mod push_down_limit; -pub mod push_down_projection; pub mod replace_distinct_aggregate; pub mod rewrite_disjunctive_predicate; pub mod scalar_subquery_to_join; diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 6967b28f3037..b54fb248a7c7 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -925,20 +925,32 @@ fn is_projection_unnecessary(input: &LogicalPlan, proj_exprs: &[Expr]) -> Result #[cfg(test)] mod tests { + use std::collections::HashMap; use std::fmt::Formatter; use std::sync::Arc; + use std::vec; use crate::optimize_projections::OptimizeProjections; + use crate::optimizer::Optimizer; use crate::test::{ - assert_optimized_plan_eq, test_table_scan, test_table_scan_with_name, + assert_fields_eq, assert_optimized_plan_eq, scan_empty, test_table_scan, + test_table_scan_fields, test_table_scan_with_name, }; + use crate::{OptimizerContext, OptimizerRule}; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{Column, DFSchemaRef, JoinType, Result, TableReference}; + use datafusion_common::{ + Column, DFSchema, DFSchemaRef, JoinType, Result, TableReference, + }; use datafusion_expr::{ - binary_expr, build_join_schema, col, count, lit, - logical_plan::builder::LogicalPlanBuilder, not, table_scan, try_cast, when, - BinaryExpr, Expr, Extension, Like, LogicalPlan, Operator, - UserDefinedLogicalNodeCore, + binary_expr, build_join_schema, + builder::table_scan_with_filters, + col, count, + expr::{self, Cast}, + lit, + logical_plan::{builder::LogicalPlanBuilder, table_scan}, + max, min, not, try_cast, when, AggregateFunction, BinaryExpr, Expr, Extension, + Like, LogicalPlan, Operator, Projection, UserDefinedLogicalNodeCore, WindowFrame, + WindowFunctionDefinition, }; fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { @@ -1466,4 +1478,617 @@ mod tests { \n TableScan: r projection=[a]"; assert_optimized_plan_equal(plan, expected) } + + #[test] + fn aggregate_no_group_by() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(Vec::::new(), vec![max(col("b"))])? + .build()?; + + let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ + \n TableScan: test projection=[b]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn aggregate_group_by() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("c")], vec![max(col("b"))])? + .build()?; + + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.b)]]\ + \n TableScan: test projection=[b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn aggregate_group_by_with_table_alias() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .alias("a")? + .aggregate(vec![col("c")], vec![max(col("b"))])? + .build()?; + + let expected = "Aggregate: groupBy=[[a.c]], aggr=[[MAX(a.b)]]\ + \n SubqueryAlias: a\ + \n TableScan: test projection=[b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn aggregate_no_group_by_with_filter() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .filter(col("c").gt(lit(1)))? + .aggregate(Vec::::new(), vec![max(col("b"))])? + .build()?; + + let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ + \n Projection: test.b\ + \n Filter: test.c > Int32(1)\ + \n TableScan: test projection=[b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn aggregate_with_periods() -> Result<()> { + let schema = Schema::new(vec![Field::new("tag.one", DataType::Utf8, false)]); + + // Build a plan that looks as follows (note "tag.one" is a column named + // "tag.one", not a column named "one" in a table named "tag"): + // + // Projection: tag.one + // Aggregate: groupBy=[], aggr=[MAX("tag.one") AS "tag.one"] + // TableScan + let plan = table_scan(Some("m4"), &schema, None)? + .aggregate( + Vec::::new(), + vec![max(col(Column::new_unqualified("tag.one"))).alias("tag.one")], + )? + .project([col(Column::new_unqualified("tag.one"))])? + .build()?; + + let expected = "\ + Aggregate: groupBy=[[]], aggr=[[MAX(m4.tag.one) AS tag.one]]\ + \n TableScan: m4 projection=[tag.one]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn redundant_project() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b"), col("c")])? + .project(vec![col("a"), col("c"), col("b")])? + .build()?; + let expected = "Projection: test.a, test.c, test.b\ + \n TableScan: test projection=[a, b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn reorder_scan() -> Result<()> { + let schema = Schema::new(test_table_scan_fields()); + + let plan = table_scan(Some("test"), &schema, Some(vec![1, 0, 2]))?.build()?; + let expected = "TableScan: test projection=[b, a, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn reorder_scan_projection() -> Result<()> { + let schema = Schema::new(test_table_scan_fields()); + + let plan = table_scan(Some("test"), &schema, Some(vec![1, 0, 2]))? + .project(vec![col("a"), col("b")])? + .build()?; + let expected = "Projection: test.a, test.b\ + \n TableScan: test projection=[b, a]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn reorder_projection() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("c"), col("b"), col("a")])? + .build()?; + let expected = "Projection: test.c, test.b, test.a\ + \n TableScan: test projection=[a, b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn noncontinuous_redundant_projection() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("c"), col("b"), col("a")])? + .filter(col("c").gt(lit(1)))? + .project(vec![col("c"), col("a"), col("b")])? + .filter(col("b").gt(lit(1)))? + .filter(col("a").gt(lit(1)))? + .project(vec![col("a"), col("c"), col("b")])? + .build()?; + let expected = "Projection: test.a, test.c, test.b\ + \n Filter: test.a > Int32(1)\ + \n Filter: test.b > Int32(1)\ + \n Projection: test.c, test.a, test.b\ + \n Filter: test.c > Int32(1)\ + \n Projection: test.c, test.b, test.a\ + \n TableScan: test projection=[a, b, c]"; + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn join_schema_trim_full_join_column_projection() -> Result<()> { + let table_scan = test_table_scan()?; + + let schema = Schema::new(vec![Field::new("c1", DataType::UInt32, false)]); + let table2_scan = scan_empty(Some("test2"), &schema, None)?.build()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .join(table2_scan, JoinType::Left, (vec!["a"], vec!["c1"]), None)? + .project(vec![col("a"), col("b"), col("c1")])? + .build()?; + + // make sure projections are pushed down to both table scans + let expected = "Left Join: test.a = test2.c1\ + \n TableScan: test projection=[a, b]\ + \n TableScan: test2 projection=[c1]"; + + let optimized_plan = optimize(plan)?; + let formatted_plan = format!("{optimized_plan:?}"); + assert_eq!(formatted_plan, expected); + + // make sure schema for join node include both join columns + let optimized_join = optimized_plan; + assert_eq!( + **optimized_join.schema(), + DFSchema::new_with_metadata( + vec![ + ( + Some("test".into()), + Arc::new(Field::new("a", DataType::UInt32, false)) + ), + ( + Some("test".into()), + Arc::new(Field::new("b", DataType::UInt32, false)) + ), + ( + Some("test2".into()), + Arc::new(Field::new("c1", DataType::UInt32, true)) + ), + ], + HashMap::new() + )?, + ); + + Ok(()) + } + + #[test] + fn join_schema_trim_partial_join_column_projection() -> Result<()> { + // test join column push down without explicit column projections + + let table_scan = test_table_scan()?; + + let schema = Schema::new(vec![Field::new("c1", DataType::UInt32, false)]); + let table2_scan = scan_empty(Some("test2"), &schema, None)?.build()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .join(table2_scan, JoinType::Left, (vec!["a"], vec!["c1"]), None)? + // projecting joined column `a` should push the right side column `c1` projection as + // well into test2 table even though `c1` is not referenced in projection. + .project(vec![col("a"), col("b")])? + .build()?; + + // make sure projections are pushed down to both table scans + let expected = "Projection: test.a, test.b\ + \n Left Join: test.a = test2.c1\ + \n TableScan: test projection=[a, b]\ + \n TableScan: test2 projection=[c1]"; + + let optimized_plan = optimize(plan)?; + let formatted_plan = format!("{optimized_plan:?}"); + assert_eq!(formatted_plan, expected); + + // make sure schema for join node include both join columns + let optimized_join = optimized_plan.inputs()[0]; + assert_eq!( + **optimized_join.schema(), + DFSchema::new_with_metadata( + vec![ + ( + Some("test".into()), + Arc::new(Field::new("a", DataType::UInt32, false)) + ), + ( + Some("test".into()), + Arc::new(Field::new("b", DataType::UInt32, false)) + ), + ( + Some("test2".into()), + Arc::new(Field::new("c1", DataType::UInt32, true)) + ), + ], + HashMap::new() + )?, + ); + + Ok(()) + } + + #[test] + fn join_schema_trim_using_join() -> Result<()> { + // shared join columns from using join should be pushed to both sides + + let table_scan = test_table_scan()?; + + let schema = Schema::new(vec![Field::new("a", DataType::UInt32, false)]); + let table2_scan = scan_empty(Some("test2"), &schema, None)?.build()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .join_using(table2_scan, JoinType::Left, vec!["a"])? + .project(vec![col("a"), col("b")])? + .build()?; + + // make sure projections are pushed down to table scan + let expected = "Projection: test.a, test.b\ + \n Left Join: Using test.a = test2.a\ + \n TableScan: test projection=[a, b]\ + \n TableScan: test2 projection=[a]"; + + let optimized_plan = optimize(plan)?; + let formatted_plan = format!("{optimized_plan:?}"); + assert_eq!(formatted_plan, expected); + + // make sure schema for join node include both join columns + let optimized_join = optimized_plan.inputs()[0]; + assert_eq!( + **optimized_join.schema(), + DFSchema::new_with_metadata( + vec![ + ( + Some("test".into()), + Arc::new(Field::new("a", DataType::UInt32, false)) + ), + ( + Some("test".into()), + Arc::new(Field::new("b", DataType::UInt32, false)) + ), + ( + Some("test2".into()), + Arc::new(Field::new("a", DataType::UInt32, true)) + ), + ], + HashMap::new() + )?, + ); + + Ok(()) + } + + #[test] + fn cast() -> Result<()> { + let table_scan = test_table_scan()?; + + let projection = LogicalPlanBuilder::from(table_scan) + .project(vec![Expr::Cast(Cast::new( + Box::new(col("c")), + DataType::Float64, + ))])? + .build()?; + + let expected = "Projection: CAST(test.c AS Float64)\ + \n TableScan: test projection=[c]"; + + assert_optimized_plan_equal(projection, expected) + } + + #[test] + fn table_scan_projected_schema() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(test_table_scan()?) + .project(vec![col("a"), col("b")])? + .build()?; + + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + assert_fields_eq(&plan, vec!["a", "b"]); + + let expected = "TableScan: test projection=[a, b]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn table_scan_projected_schema_non_qualified_relation() -> Result<()> { + let table_scan = test_table_scan()?; + let input_schema = table_scan.schema(); + assert_eq!(3, input_schema.fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + + // Build the LogicalPlan directly (don't use PlanBuilder), so + // that the Column references are unqualified (e.g. their + // relation is `None`). PlanBuilder resolves the expressions + let expr = vec![col("test.a"), col("test.b")]; + let plan = + LogicalPlan::Projection(Projection::try_new(expr, Arc::new(table_scan))?); + + assert_fields_eq(&plan, vec!["a", "b"]); + + let expected = "TableScan: test projection=[a, b]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn table_limit() -> Result<()> { + let table_scan = test_table_scan()?; + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("c"), col("a")])? + .limit(0, Some(5))? + .build()?; + + assert_fields_eq(&plan, vec!["c", "a"]); + + let expected = "Limit: skip=0, fetch=5\ + \n Projection: test.c, test.a\ + \n TableScan: test projection=[a, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn table_scan_without_projection() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan).build()?; + // should expand projection to all columns without projection + let expected = "TableScan: test projection=[a, b, c]"; + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn table_scan_with_literal_projection() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![lit(1_i64), lit(2_i64)])? + .build()?; + let expected = "Projection: Int64(1), Int64(2)\ + \n TableScan: test projection=[]"; + assert_optimized_plan_equal(plan, expected) + } + + /// tests that it removes unused columns in projections + #[test] + fn table_unused_column() -> Result<()> { + let table_scan = test_table_scan()?; + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + + // we never use "b" in the first projection => remove it + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("c"), col("a"), col("b")])? + .filter(col("c").gt(lit(1)))? + .aggregate(vec![col("c")], vec![max(col("a"))])? + .build()?; + + assert_fields_eq(&plan, vec!["c", "MAX(test.a)"]); + + let plan = optimize(plan).expect("failed to optimize plan"); + let expected = "\ + Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.a)]]\ + \n Filter: test.c > Int32(1)\ + \n Projection: test.c, test.a\ + \n TableScan: test projection=[a, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + /// tests that it removes un-needed projections + #[test] + fn table_unused_projection() -> Result<()> { + let table_scan = test_table_scan()?; + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + + // there is no need for the first projection + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("b")])? + .project(vec![lit(1).alias("a")])? + .build()?; + + assert_fields_eq(&plan, vec!["a"]); + + let expected = "\ + Projection: Int32(1) AS a\ + \n TableScan: test projection=[]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn table_full_filter_pushdown() -> Result<()> { + let schema = Schema::new(test_table_scan_fields()); + + let table_scan = table_scan_with_filters( + Some("test"), + &schema, + None, + vec![col("b").eq(lit(1))], + )? + .build()?; + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + + // there is no need for the first projection + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("b")])? + .project(vec![lit(1).alias("a")])? + .build()?; + + assert_fields_eq(&plan, vec!["a"]); + + let expected = "\ + Projection: Int32(1) AS a\ + \n TableScan: test projection=[], full_filters=[b = Int32(1)]"; + + assert_optimized_plan_equal(plan, expected) + } + + /// tests that optimizing twice yields same plan + #[test] + fn test_double_optimization() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("b")])? + .project(vec![lit(1).alias("a")])? + .build()?; + + let optimized_plan1 = optimize(plan).expect("failed to optimize plan"); + let optimized_plan2 = + optimize(optimized_plan1.clone()).expect("failed to optimize plan"); + + let formatted_plan1 = format!("{optimized_plan1:?}"); + let formatted_plan2 = format!("{optimized_plan2:?}"); + assert_eq!(formatted_plan1, formatted_plan2); + Ok(()) + } + + /// tests that it removes an aggregate is never used downstream + #[test] + fn table_unused_aggregate() -> Result<()> { + let table_scan = test_table_scan()?; + assert_eq!(3, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a", "b", "c"]); + + // we never use "min(b)" => remove it + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("a"), col("c")], vec![max(col("b")), min(col("b"))])? + .filter(col("c").gt(lit(1)))? + .project(vec![col("c"), col("a"), col("MAX(test.b)")])? + .build()?; + + assert_fields_eq(&plan, vec!["c", "a", "MAX(test.b)"]); + + let expected = "Projection: test.c, test.a, MAX(test.b)\ + \n Filter: test.c > Int32(1)\ + \n Aggregate: groupBy=[[test.a, test.c]], aggr=[[MAX(test.b)]]\ + \n TableScan: test projection=[a, b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn aggregate_filter_pushdown() -> Result<()> { + let table_scan = test_table_scan()?; + + let aggr_with_filter = Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Count, + vec![col("b")], + false, + Some(Box::new(col("c").gt(lit(42)))), + None, + None, + )); + + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate( + vec![col("a")], + vec![count(col("b")), aggr_with_filter.alias("count2")], + )? + .build()?; + + let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.b), COUNT(test.b) FILTER (WHERE test.c > Int32(42)) AS count2]]\ + \n TableScan: test projection=[a, b, c]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn pushdown_through_distinct() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b")])? + .distinct()? + .project(vec![col("a")])? + .build()?; + + let expected = "Projection: test.a\ + \n Distinct:\ + \n TableScan: test projection=[a, b]"; + + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn test_window() -> Result<()> { + let table_scan = test_table_scan()?; + + let max1 = Expr::WindowFunction(expr::WindowFunction::new( + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), + vec![col("test.a")], + vec![col("test.b")], + vec![], + WindowFrame::new(None), + None, + )); + + let max2 = Expr::WindowFunction(expr::WindowFunction::new( + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), + vec![col("test.b")], + vec![], + vec![], + WindowFrame::new(None), + None, + )); + let col1 = col(max1.display_name()?); + let col2 = col(max2.display_name()?); + + let plan = LogicalPlanBuilder::from(table_scan) + .window(vec![max1])? + .window(vec![max2])? + .project(vec![col1, col2])? + .build()?; + + let expected = "Projection: MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MAX(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[MAX(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + \n Projection: test.b, MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + \n TableScan: test projection=[a, b]"; + + assert_optimized_plan_equal(plan, expected) + } + + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + + fn optimize(plan: LogicalPlan) -> Result { + let optimizer = Optimizer::with_rules(vec![Arc::new(OptimizeProjections::new())]); + let optimized_plan = + optimizer.optimize(plan, &OptimizerContext::new(), observe)?; + Ok(optimized_plan) + } } diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs deleted file mode 100644 index 2f578094b3bc..000000000000 --- a/datafusion/optimizer/src/push_down_projection.rs +++ /dev/null @@ -1,660 +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. - -#[cfg(test)] -mod tests { - use std::collections::HashMap; - use std::sync::Arc; - use std::vec; - - use crate::optimize_projections::OptimizeProjections; - use crate::optimizer::Optimizer; - use crate::test::*; - use crate::{OptimizerContext, OptimizerRule}; - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{Column, DFSchema, Result}; - use datafusion_expr::builder::table_scan_with_filters; - use datafusion_expr::expr::{self, Cast}; - use datafusion_expr::logical_plan::{ - builder::LogicalPlanBuilder, table_scan, JoinType, - }; - use datafusion_expr::{ - col, count, lit, max, min, AggregateFunction, Expr, LogicalPlan, Projection, - WindowFrame, WindowFunctionDefinition, - }; - - #[test] - fn aggregate_no_group_by() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .aggregate(Vec::::new(), vec![max(col("b"))])? - .build()?; - - let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ - \n TableScan: test projection=[b]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn aggregate_group_by() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .aggregate(vec![col("c")], vec![max(col("b"))])? - .build()?; - - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.b)]]\ - \n TableScan: test projection=[b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn aggregate_group_by_with_table_alias() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .alias("a")? - .aggregate(vec![col("c")], vec![max(col("b"))])? - .build()?; - - let expected = "Aggregate: groupBy=[[a.c]], aggr=[[MAX(a.b)]]\ - \n SubqueryAlias: a\ - \n TableScan: test projection=[b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn aggregate_no_group_by_with_filter() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .filter(col("c").gt(lit(1)))? - .aggregate(Vec::::new(), vec![max(col("b"))])? - .build()?; - - let expected = "Aggregate: groupBy=[[]], aggr=[[MAX(test.b)]]\ - \n Projection: test.b\ - \n Filter: test.c > Int32(1)\ - \n TableScan: test projection=[b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn aggregate_with_periods() -> Result<()> { - let schema = Schema::new(vec![Field::new("tag.one", DataType::Utf8, false)]); - - // Build a plan that looks as follows (note "tag.one" is a column named - // "tag.one", not a column named "one" in a table named "tag"): - // - // Projection: tag.one - // Aggregate: groupBy=[], aggr=[MAX("tag.one") AS "tag.one"] - // TableScan - let plan = table_scan(Some("m4"), &schema, None)? - .aggregate( - Vec::::new(), - vec![max(col(Column::new_unqualified("tag.one"))).alias("tag.one")], - )? - .project([col(Column::new_unqualified("tag.one"))])? - .build()?; - - let expected = "\ - Aggregate: groupBy=[[]], aggr=[[MAX(m4.tag.one) AS tag.one]]\ - \n TableScan: m4 projection=[tag.one]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn redundant_project() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .project(vec![col("a"), col("c"), col("b")])? - .build()?; - let expected = "Projection: test.a, test.c, test.b\ - \n TableScan: test projection=[a, b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn reorder_scan() -> Result<()> { - let schema = Schema::new(test_table_scan_fields()); - - let plan = table_scan(Some("test"), &schema, Some(vec![1, 0, 2]))?.build()?; - let expected = "TableScan: test projection=[b, a, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn reorder_scan_projection() -> Result<()> { - let schema = Schema::new(test_table_scan_fields()); - - let plan = table_scan(Some("test"), &schema, Some(vec![1, 0, 2]))? - .project(vec![col("a"), col("b")])? - .build()?; - let expected = "Projection: test.a, test.b\ - \n TableScan: test projection=[b, a]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn reorder_projection() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("c"), col("b"), col("a")])? - .build()?; - let expected = "Projection: test.c, test.b, test.a\ - \n TableScan: test projection=[a, b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn noncontinuous_redundant_projection() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("c"), col("b"), col("a")])? - .filter(col("c").gt(lit(1)))? - .project(vec![col("c"), col("a"), col("b")])? - .filter(col("b").gt(lit(1)))? - .filter(col("a").gt(lit(1)))? - .project(vec![col("a"), col("c"), col("b")])? - .build()?; - let expected = "Projection: test.a, test.c, test.b\ - \n Filter: test.a > Int32(1)\ - \n Filter: test.b > Int32(1)\ - \n Projection: test.c, test.a, test.b\ - \n Filter: test.c > Int32(1)\ - \n Projection: test.c, test.b, test.a\ - \n TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn join_schema_trim_full_join_column_projection() -> Result<()> { - let table_scan = test_table_scan()?; - - let schema = Schema::new(vec![Field::new("c1", DataType::UInt32, false)]); - let table2_scan = scan_empty(Some("test2"), &schema, None)?.build()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .join(table2_scan, JoinType::Left, (vec!["a"], vec!["c1"]), None)? - .project(vec![col("a"), col("b"), col("c1")])? - .build()?; - - // make sure projections are pushed down to both table scans - let expected = "Left Join: test.a = test2.c1\ - \n TableScan: test projection=[a, b]\ - \n TableScan: test2 projection=[c1]"; - - let optimized_plan = optimize(plan)?; - let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(formatted_plan, expected); - - // make sure schema for join node include both join columns - let optimized_join = optimized_plan; - assert_eq!( - **optimized_join.schema(), - DFSchema::new_with_metadata( - vec![ - ( - Some("test".into()), - Arc::new(Field::new("a", DataType::UInt32, false)) - ), - ( - Some("test".into()), - Arc::new(Field::new("b", DataType::UInt32, false)) - ), - ( - Some("test2".into()), - Arc::new(Field::new("c1", DataType::UInt32, true)) - ), - ], - HashMap::new() - )?, - ); - - Ok(()) - } - - #[test] - fn join_schema_trim_partial_join_column_projection() -> Result<()> { - // test join column push down without explicit column projections - - let table_scan = test_table_scan()?; - - let schema = Schema::new(vec![Field::new("c1", DataType::UInt32, false)]); - let table2_scan = scan_empty(Some("test2"), &schema, None)?.build()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .join(table2_scan, JoinType::Left, (vec!["a"], vec!["c1"]), None)? - // projecting joined column `a` should push the right side column `c1` projection as - // well into test2 table even though `c1` is not referenced in projection. - .project(vec![col("a"), col("b")])? - .build()?; - - // make sure projections are pushed down to both table scans - let expected = "Projection: test.a, test.b\ - \n Left Join: test.a = test2.c1\ - \n TableScan: test projection=[a, b]\ - \n TableScan: test2 projection=[c1]"; - - let optimized_plan = optimize(plan)?; - let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(formatted_plan, expected); - - // make sure schema for join node include both join columns - let optimized_join = optimized_plan.inputs()[0]; - assert_eq!( - **optimized_join.schema(), - DFSchema::new_with_metadata( - vec![ - ( - Some("test".into()), - Arc::new(Field::new("a", DataType::UInt32, false)) - ), - ( - Some("test".into()), - Arc::new(Field::new("b", DataType::UInt32, false)) - ), - ( - Some("test2".into()), - Arc::new(Field::new("c1", DataType::UInt32, true)) - ), - ], - HashMap::new() - )?, - ); - - Ok(()) - } - - #[test] - fn join_schema_trim_using_join() -> Result<()> { - // shared join columns from using join should be pushed to both sides - - let table_scan = test_table_scan()?; - - let schema = Schema::new(vec![Field::new("a", DataType::UInt32, false)]); - let table2_scan = scan_empty(Some("test2"), &schema, None)?.build()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .join_using(table2_scan, JoinType::Left, vec!["a"])? - .project(vec![col("a"), col("b")])? - .build()?; - - // make sure projections are pushed down to table scan - let expected = "Projection: test.a, test.b\ - \n Left Join: Using test.a = test2.a\ - \n TableScan: test projection=[a, b]\ - \n TableScan: test2 projection=[a]"; - - let optimized_plan = optimize(plan)?; - let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(formatted_plan, expected); - - // make sure schema for join node include both join columns - let optimized_join = optimized_plan.inputs()[0]; - assert_eq!( - **optimized_join.schema(), - DFSchema::new_with_metadata( - vec![ - ( - Some("test".into()), - Arc::new(Field::new("a", DataType::UInt32, false)) - ), - ( - Some("test".into()), - Arc::new(Field::new("b", DataType::UInt32, false)) - ), - ( - Some("test2".into()), - Arc::new(Field::new("a", DataType::UInt32, true)) - ), - ], - HashMap::new() - )?, - ); - - Ok(()) - } - - #[test] - fn cast() -> Result<()> { - let table_scan = test_table_scan()?; - - let projection = LogicalPlanBuilder::from(table_scan) - .project(vec![Expr::Cast(Cast::new( - Box::new(col("c")), - DataType::Float64, - ))])? - .build()?; - - let expected = "Projection: CAST(test.c AS Float64)\ - \n TableScan: test projection=[c]"; - - assert_optimized_plan_eq(projection, expected) - } - - #[test] - fn table_scan_projected_schema() -> Result<()> { - let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(test_table_scan()?) - .project(vec![col("a"), col("b")])? - .build()?; - - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - assert_fields_eq(&plan, vec!["a", "b"]); - - let expected = "TableScan: test projection=[a, b]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn table_scan_projected_schema_non_qualified_relation() -> Result<()> { - let table_scan = test_table_scan()?; - let input_schema = table_scan.schema(); - assert_eq!(3, input_schema.fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - // Build the LogicalPlan directly (don't use PlanBuilder), so - // that the Column references are unqualified (e.g. their - // relation is `None`). PlanBuilder resolves the expressions - let expr = vec![col("test.a"), col("test.b")]; - let plan = - LogicalPlan::Projection(Projection::try_new(expr, Arc::new(table_scan))?); - - assert_fields_eq(&plan, vec!["a", "b"]); - - let expected = "TableScan: test projection=[a, b]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn table_limit() -> Result<()> { - let table_scan = test_table_scan()?; - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("c"), col("a")])? - .limit(0, Some(5))? - .build()?; - - assert_fields_eq(&plan, vec!["c", "a"]); - - let expected = "Limit: skip=0, fetch=5\ - \n Projection: test.c, test.a\ - \n TableScan: test projection=[a, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn table_scan_without_projection() -> Result<()> { - let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan).build()?; - // should expand projection to all columns without projection - let expected = "TableScan: test projection=[a, b, c]"; - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn table_scan_with_literal_projection() -> Result<()> { - let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![lit(1_i64), lit(2_i64)])? - .build()?; - let expected = "Projection: Int64(1), Int64(2)\ - \n TableScan: test projection=[]"; - assert_optimized_plan_eq(plan, expected) - } - - /// tests that it removes unused columns in projections - #[test] - fn table_unused_column() -> Result<()> { - let table_scan = test_table_scan()?; - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - // we never use "b" in the first projection => remove it - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("c"), col("a"), col("b")])? - .filter(col("c").gt(lit(1)))? - .aggregate(vec![col("c")], vec![max(col("a"))])? - .build()?; - - assert_fields_eq(&plan, vec!["c", "MAX(test.a)"]); - - let plan = optimize(plan).expect("failed to optimize plan"); - let expected = "\ - Aggregate: groupBy=[[test.c]], aggr=[[MAX(test.a)]]\ - \n Filter: test.c > Int32(1)\ - \n Projection: test.c, test.a\ - \n TableScan: test projection=[a, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - /// tests that it removes un-needed projections - #[test] - fn table_unused_projection() -> Result<()> { - let table_scan = test_table_scan()?; - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - // there is no need for the first projection - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("b")])? - .project(vec![lit(1).alias("a")])? - .build()?; - - assert_fields_eq(&plan, vec!["a"]); - - let expected = "\ - Projection: Int32(1) AS a\ - \n TableScan: test projection=[]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn table_full_filter_pushdown() -> Result<()> { - let schema = Schema::new(test_table_scan_fields()); - - let table_scan = table_scan_with_filters( - Some("test"), - &schema, - None, - vec![col("b").eq(lit(1))], - )? - .build()?; - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - // there is no need for the first projection - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("b")])? - .project(vec![lit(1).alias("a")])? - .build()?; - - assert_fields_eq(&plan, vec!["a"]); - - let expected = "\ - Projection: Int32(1) AS a\ - \n TableScan: test projection=[], full_filters=[b = Int32(1)]"; - - assert_optimized_plan_eq(plan, expected) - } - - /// tests that optimizing twice yields same plan - #[test] - fn test_double_optimization() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("b")])? - .project(vec![lit(1).alias("a")])? - .build()?; - - let optimized_plan1 = optimize(plan).expect("failed to optimize plan"); - let optimized_plan2 = - optimize(optimized_plan1.clone()).expect("failed to optimize plan"); - - let formatted_plan1 = format!("{optimized_plan1:?}"); - let formatted_plan2 = format!("{optimized_plan2:?}"); - assert_eq!(formatted_plan1, formatted_plan2); - Ok(()) - } - - /// tests that it removes an aggregate is never used downstream - #[test] - fn table_unused_aggregate() -> Result<()> { - let table_scan = test_table_scan()?; - assert_eq!(3, table_scan.schema().fields().len()); - assert_fields_eq(&table_scan, vec!["a", "b", "c"]); - - // we never use "min(b)" => remove it - let plan = LogicalPlanBuilder::from(table_scan) - .aggregate(vec![col("a"), col("c")], vec![max(col("b")), min(col("b"))])? - .filter(col("c").gt(lit(1)))? - .project(vec![col("c"), col("a"), col("MAX(test.b)")])? - .build()?; - - assert_fields_eq(&plan, vec!["c", "a", "MAX(test.b)"]); - - let expected = "Projection: test.c, test.a, MAX(test.b)\ - \n Filter: test.c > Int32(1)\ - \n Aggregate: groupBy=[[test.a, test.c]], aggr=[[MAX(test.b)]]\ - \n TableScan: test projection=[a, b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn aggregate_filter_pushdown() -> Result<()> { - let table_scan = test_table_scan()?; - - let aggr_with_filter = Expr::AggregateFunction(expr::AggregateFunction::new( - AggregateFunction::Count, - vec![col("b")], - false, - Some(Box::new(col("c").gt(lit(42)))), - None, - None, - )); - - let plan = LogicalPlanBuilder::from(table_scan) - .aggregate( - vec![col("a")], - vec![count(col("b")), aggr_with_filter.alias("count2")], - )? - .build()?; - - let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(test.b), COUNT(test.b) FILTER (WHERE test.c > Int32(42)) AS count2]]\ - \n TableScan: test projection=[a, b, c]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn pushdown_through_distinct() -> Result<()> { - let table_scan = test_table_scan()?; - - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b")])? - .distinct()? - .project(vec![col("a")])? - .build()?; - - let expected = "Projection: test.a\ - \n Distinct:\ - \n TableScan: test projection=[a, b]"; - - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn test_window() -> Result<()> { - let table_scan = test_table_scan()?; - - let max1 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), - vec![col("test.a")], - vec![col("test.b")], - vec![], - WindowFrame::new(None), - None, - )); - - let max2 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), - vec![col("test.b")], - vec![], - vec![], - WindowFrame::new(None), - None, - )); - let col1 = col(max1.display_name()?); - let col2 = col(max2.display_name()?); - - let plan = LogicalPlanBuilder::from(table_scan) - .window(vec![max1])? - .window(vec![max2])? - .project(vec![col1, col2])? - .build()?; - - let expected = "Projection: MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MAX(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(test.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n Projection: test.b, MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[MAX(test.a) PARTITION BY [test.b] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n TableScan: test projection=[a, b]"; - - assert_optimized_plan_eq(plan, expected) - } - - fn assert_optimized_plan_eq(plan: LogicalPlan, expected: &str) -> Result<()> { - let optimized_plan = optimize(plan).expect("failed to optimize plan"); - let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(formatted_plan, expected); - Ok(()) - } - - fn optimize(plan: LogicalPlan) -> Result { - let optimizer = Optimizer::with_rules(vec![Arc::new(OptimizeProjections::new())]); - let optimized_plan = - optimizer.optimize(plan, &OptimizerContext::new(), observe)?; - - Ok(optimized_plan) - } - - fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} -} From 4e9f2d5260ecaca261076d2e0a626d3f595ff5a5 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Mon, 15 Apr 2024 21:13:07 +0800 Subject: [PATCH 39/39] Move conversion of FIRST/LAST Aggregate function to independent physical optimizer rule (#10061) * move out the ordering ruel Signed-off-by: jayzhan211 * introduce rule Signed-off-by: jayzhan211 * revert test result Signed-off-by: jayzhan211 * pass mulit order test Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * with new childes Signed-off-by: jayzhan211 * revert slt Signed-off-by: jayzhan211 * revert back Signed-off-by: jayzhan211 * rm rewrite in new child Signed-off-by: jayzhan211 * backup Signed-off-by: jayzhan211 * only move conversion to optimizer Signed-off-by: jayzhan211 * find test that do reverse Signed-off-by: jayzhan211 * add test for first and last Signed-off-by: jayzhan211 * pass all test Signed-off-by: jayzhan211 * upd test Signed-off-by: jayzhan211 * upd test Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * add aggregate test Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * final draft Signed-off-by: jayzhan211 * cleanup again Signed-off-by: jayzhan211 * pull out finer ordering code and reuse Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 * remove finer in optimize rule Signed-off-by: jayzhan211 * add comments and clenaup Signed-off-by: jayzhan211 * rename fun Signed-off-by: jayzhan211 * rename fun Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * avoid unnecessary recursion and rename Signed-off-by: jayzhan211 * remove unnecessary rule Signed-off-by: jayzhan211 * fix merge Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical_optimizer/convert_first_last.rs | 260 ++++++++++++++++++ datafusion/core/src/physical_optimizer/mod.rs | 1 + .../core/src/physical_optimizer/optimizer.rs | 3 + .../core/tests/data/convert_first_last.csv | 11 + .../physical-plan/src/aggregates/mod.rs | 117 ++++---- datafusion/physical-plan/src/tree_node.rs | 1 + datafusion/physical-plan/src/windows/mod.rs | 2 +- .../sqllogictest/test_files/aggregate.slt | 42 +++ .../sqllogictest/test_files/explain.slt | 3 + .../sqllogictest/test_files/group_by.slt | 4 +- 10 files changed, 372 insertions(+), 72 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/convert_first_last.rs create mode 100644 datafusion/core/tests/data/convert_first_last.csv diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs new file mode 100644 index 000000000000..4102313d3126 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -0,0 +1,260 @@ +// 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 datafusion_common::Result; +use datafusion_common::{ + config::ConfigOptions, + tree_node::{Transformed, TransformedResult, TreeNode}, +}; +use datafusion_physical_expr::expressions::{FirstValue, LastValue}; +use datafusion_physical_expr::{ + equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, + EquivalenceProperties, PhysicalSortRequirement, +}; +use datafusion_physical_plan::aggregates::concat_slices; +use datafusion_physical_plan::{ + aggregates::{AggregateExec, AggregateMode}, + ExecutionPlan, ExecutionPlanProperties, InputOrderMode, +}; +use std::sync::Arc; + +use datafusion_physical_plan::windows::get_ordered_partition_by_indices; + +use super::PhysicalOptimizerRule; + +/// The optimizer rule check the ordering requirements of the aggregate expressions. +/// And convert between FIRST_VALUE and LAST_VALUE if possible. +/// For example, If we have an ascending values and we want LastValue from the descending requirement, +/// it is equivalent to FirstValue with the current ascending ordering. +/// +/// The concrete example is that, says we have values c1 with [1, 2, 3], which is an ascending order. +/// If we want LastValue(c1 order by desc), which is the first value of reversed c1 [3, 2, 1], +/// so we can convert the aggregate expression to FirstValue(c1 order by asc), +/// since the current ordering is already satisfied, it saves our time! +#[derive(Default)] +pub struct OptimizeAggregateOrder {} + +impl OptimizeAggregateOrder { + pub fn new() -> Self { + Self::default() + } +} + +impl PhysicalOptimizerRule for OptimizeAggregateOrder { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_up(&get_common_requirement_of_aggregate_input) + .data() + } + + fn name(&self) -> &str { + "OptimizeAggregateOrder" + } + + fn schema_check(&self) -> bool { + true + } +} + +fn get_common_requirement_of_aggregate_input( + plan: Arc, +) -> Result>> { + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let input = aggr_exec.input(); + let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); + let group_by = aggr_exec.group_by(); + let mode = aggr_exec.mode(); + + let input_eq_properties = input.equivalence_properties(); + let groupby_exprs = group_by.input_exprs(); + // If existing ordering satisfies a prefix of the GROUP BY expressions, + // prefix requirements with this section. In this case, aggregation will + // work more efficiently. + let indices = get_ordered_partition_by_indices(&groupby_exprs, input); + let requirement = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + + try_convert_first_last_if_better( + &requirement, + &mut aggr_expr, + input_eq_properties, + )?; + + let required_input_ordering = (!requirement.is_empty()).then_some(requirement); + + let input_order_mode = + if indices.len() == groupby_exprs.len() && !indices.is_empty() { + InputOrderMode::Sorted + } else if !indices.is_empty() { + InputOrderMode::PartiallySorted(indices) + } else { + InputOrderMode::Linear + }; + let projection_mapping = + ProjectionMapping::try_new(group_by.expr(), &input.schema())?; + + let cache = AggregateExec::compute_properties( + input, + plan.schema().clone(), + &projection_mapping, + mode, + &input_order_mode, + ); + + let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info( + required_input_ordering, + aggr_expr, + cache, + input_order_mode, + ); + + Ok(Transformed::yes( + Arc::new(aggr_exec) as Arc + )) + } else { + Ok(Transformed::no(plan)) + } +} + +/// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer +/// is in Partial mode and the second layer is in Final or Finalpartitioned mode. +/// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. +/// Therefore, we check it and get the updated aggregate expressions. +/// +/// If AggregateExec is created from elsewhere, we skip the check and return the original aggregate expressions. +fn try_get_updated_aggr_expr_from_child( + aggr_exec: &AggregateExec, +) -> Vec> { + let input = aggr_exec.input(); + if aggr_exec.mode() == &AggregateMode::Final + || aggr_exec.mode() == &AggregateMode::FinalPartitioned + { + // Some aggregators may be modified during initialization for + // optimization purposes. For example, a FIRST_VALUE may turn + // into a LAST_VALUE with the reverse ordering requirement. + // To reflect such changes to subsequent stages, use the updated + // `AggregateExpr`/`PhysicalSortExpr` objects. + // + // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] + if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { + if c_aggr_exec.mode() == &AggregateMode::Partial { + // If the input is an AggregateExec in Partial mode, then the + // input is a CoalescePartitionsExec. In this case, the + // AggregateExec is the second stage of aggregation. The + // requirements of the second stage are the requirements of + // the first stage. + return c_aggr_exec.aggr_expr().to_vec(); + } + } + } + + aggr_exec.aggr_expr().to_vec() +} + +/// Get the common requirement that satisfies all the aggregate expressions. +/// +/// # Parameters +/// +/// - `aggr_exprs`: A slice of `Arc` containing all the +/// aggregate expressions. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `eq_properties`: A reference to an `EquivalenceProperties` instance +/// representing equivalence properties for ordering. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexRequirement` instance, which is the requirement that satisfies all the +/// aggregate requirements. Returns an error in case of conflicting requirements. +/// +/// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this +/// function care only the possible conversion between FIRST_VALUE and LAST_VALUE +fn try_convert_first_last_if_better( + prefix_requirement: &[PhysicalSortRequirement], + aggr_exprs: &mut [Arc], + eq_properties: &EquivalenceProperties, +) -> Result<()> { + for aggr_expr in aggr_exprs.iter_mut() { + let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); + let reverse_aggr_req = reverse_order_bys(aggr_req); + let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); + let reverse_aggr_req = + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + + if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { + let mut first_value = first_value.clone(); + + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to LAST_VALUE enables more efficient execution + // given the existing ordering: + let mut last_value = first_value.convert_to_last(); + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + first_value = first_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(first_value) as _; + } + continue; + } + if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { + let mut last_value = last_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to FIRST_VALUE enables more efficient execution + // given the existing ordering: + let mut first_value = last_value.convert_to_first(); + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + last_value = last_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(last_value) as _; + } + continue; + } + } + + Ok(()) +} diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index e990fead610d..c80668c6da74 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -24,6 +24,7 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; +mod convert_first_last; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 48da68cb2e37..08cbf68fa617 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,6 +19,7 @@ use std::sync::Arc; +use super::convert_first_last::OptimizeAggregateOrder; use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; @@ -101,6 +102,8 @@ impl PhysicalOptimizer { // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. Arc::new(EnforceSorting::new()), + // Run once after the local sorting requirement is changed + Arc::new(OptimizeAggregateOrder::new()), // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. Arc::new(ProjectionPushdown::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the diff --git a/datafusion/core/tests/data/convert_first_last.csv b/datafusion/core/tests/data/convert_first_last.csv new file mode 100644 index 000000000000..059b631e5711 --- /dev/null +++ b/datafusion/core/tests/data/convert_first_last.csv @@ -0,0 +1,11 @@ +c1,c2,c3 +1,9,0 +2,8,1 +3,7,2 +4,6,3 +5,5,4 +6,4,5 +7,3,6 +8,2,7 +9,1,8 +10,0,9 \ No newline at end of file diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 98c44e23c6c7..ba9a6b1be0ef 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -39,12 +39,15 @@ use datafusion_common::stats::Precision; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; +use datafusion_physical_expr::aggregate::is_order_sensitive; +use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ - aggregate::is_order_sensitive, - equivalence::{collapse_lex_req, ProjectionMapping}, - expressions::{Column, FirstValue, LastValue, Max, Min, UnKnownColumn}, - physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, - LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortRequirement, + equivalence::ProjectionMapping, + expressions::{Column, Max, Min, UnKnownColumn}, + AggregateExpr, LexRequirement, PhysicalExpr, +}; +use datafusion_physical_expr::{ + physical_exprs_contains, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; use itertools::Itertools; @@ -269,6 +272,36 @@ pub struct AggregateExec { } impl AggregateExec { + /// Function used in `ConvertFirstLast` optimizer rule, + /// where we need parts of the new value, others cloned from the old one + pub fn new_with_aggr_expr_and_ordering_info( + &self, + required_input_ordering: Option, + aggr_expr: Vec>, + cache: PlanProperties, + input_order_mode: InputOrderMode, + ) -> Self { + Self { + aggr_expr, + required_input_ordering, + metrics: ExecutionPlanMetricsSet::new(), + input_order_mode, + cache, + // clone the rest of the fields + mode: self.mode, + group_by: self.group_by.clone(), + filter_expr: self.filter_expr.clone(), + limit: self.limit, + input: self.input.clone(), + schema: self.schema.clone(), + input_schema: self.input_schema.clone(), + } + } + + pub fn cache(&self) -> &PlanProperties { + &self.cache + } + /// Create a new hash aggregate execution plan pub fn try_new( mode: AggregateMode, @@ -336,8 +369,7 @@ impl AggregateExec { }) .collect::>(); - let req = get_aggregate_exprs_requirement( - &new_requirement, + let req = get_finer_aggregate_exprs_requirement( &mut aggr_expr, &group_by, input_eq_properties, @@ -369,6 +401,7 @@ impl AggregateExec { &mode, &input_order_mode, ); + Ok(AggregateExec { mode, group_by, @@ -507,7 +540,7 @@ impl AggregateExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( + pub fn compute_properties( input: &Arc, schema: SchemaRef, projection_mapping: &ProjectionMapping, @@ -683,9 +716,9 @@ impl ExecutionPlan for AggregateExec { children[0].clone(), self.input_schema.clone(), self.schema.clone(), - //self.original_schema.clone(), )?; me.limit = self.limit; + Ok(Arc::new(me)) } @@ -870,7 +903,7 @@ fn finer_ordering( } /// Concatenates the given slices. -fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { +pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { [lhs, rhs].concat() } @@ -891,8 +924,7 @@ fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. -fn get_aggregate_exprs_requirement( - prefix_requirement: &[PhysicalSortRequirement], +fn get_finer_aggregate_exprs_requirement( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, @@ -900,60 +932,6 @@ fn get_aggregate_exprs_requirement( ) -> Result { let mut requirement = vec![]; for aggr_expr in aggr_exprs.iter_mut() { - let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_req = reverse_order_bys(aggr_req); - let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - - if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { - let mut first_value = first_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to LAST_VALUE enables more efficient execution - // given the existing ordering: - let mut last_value = first_value.convert_to_last(); - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; - } - continue; - } - if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { - let mut last_value = last_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to FIRST_VALUE enables more efficient execution - // given the existing ordering: - let mut first_value = last_value.convert_to_first(); - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - last_value = last_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(last_value) as _; - } - continue; - } if let Some(finer_ordering) = finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) { @@ -1003,6 +981,7 @@ fn get_aggregate_exprs_requirement( continue; } } + // Neither the existing requirement and current aggregate requirement satisfy the other, this means // requirements are conflicting. Currently, we do not support // conflicting requirements. @@ -1010,6 +989,7 @@ fn get_aggregate_exprs_requirement( "Conflicting ordering requirements in aggregate functions is not supported" ); } + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } @@ -1235,7 +1215,7 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, LastValue, Median, OrderSensitiveArrayAgg, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, }; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, @@ -2171,8 +2151,7 @@ mod tests { }) .collect::>(); let group_by = PhysicalGroupBy::new_single(vec![]); - let res = get_aggregate_exprs_requirement( - &[], + let res = get_finer_aggregate_exprs_requirement( &mut aggr_exprs, &group_by, &eq_properties, diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs index 52a52f81bdaf..46460cbb6684 100644 --- a/datafusion/physical-plan/src/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -64,6 +64,7 @@ impl PlanContext { pub fn update_plan_from_children(mut self) -> Result { let children_plans = self.children.iter().map(|c| c.plan.clone()).collect(); self.plan = with_new_children_if_necessary(self.plan, children_plans)?; + Ok(self) } } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index c5c845614c7b..e01ee06a12b8 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -373,7 +373,7 @@ pub(crate) fn calc_requirements< /// For instance, if input is ordered by a, b, c and PARTITION BY b, a is used, /// this vector will be [1, 0]. It means that when we iterate b, a columns with the order [1, 0] /// resulting vector (a, b) is a preset of the existing ordering (a, b, c). -pub(crate) fn get_ordered_partition_by_indices( +pub fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 30d5c7243f26..3d24fe3888d7 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3455,3 +3455,45 @@ SELECT LAST_VALUE(column1 ORDER BY column2 DESC) IGNORE NULLS FROM t; statement ok DROP TABLE t; + +# Test Convert FirstLast optimizer rule +statement ok +CREATE EXTERNAL TABLE convert_first_last_table ( +c1 INT NOT NULL, +c2 INT NOT NULL, +c3 INT NOT NULL +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (c1 ASC) +WITH ORDER (c2 DESC) +WITH ORDER (c3 ASC) +LOCATION '../core/tests/data/convert_first_last.csv'; + +# test first to last, the result does not show difference, we need to check the conversion by `explain` +query TT +explain select first_value(c1 order by c3 desc) from convert_first_last_table; +---- +logical_plan +01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] +02)--TableScan: convert_first_last_table projection=[c1, c3] +physical_plan +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true + +# test last to first +query TT +explain select last_value(c1 order by c2 asc) from convert_first_last_table; +---- +logical_plan +01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] +02)--TableScan: convert_first_last_table projection=[c1, c2] +physical_plan +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index c357391e70b5..d8c8dcd41b6a 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -248,6 +248,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true @@ -304,6 +305,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements @@ -340,6 +342,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 1acdcde9c8ee..5c5bf58dd049 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2805,7 +2805,7 @@ logical_plan 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] 03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -3800,7 +3800,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)]